You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@skywalking.apache.org by ke...@apache.org on 2021/08/31 07:40:34 UTC

[skywalking] 01/01: Rebuilt ElasticSearch client on top of their REST API Remove page path from browser log query condition Add Python Falcon Plugin Logo (#528)

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

kezhenxu94 pushed a commit to branch feature/elasticsearch-client
in repository https://gitbox.apache.org/repos/asf/skywalking.git

commit ad4f740b23b7b2a3949777fbeab78b95b7551556
Author: kezhenxu94 <ke...@apache.org>
AuthorDate: Sat Aug 21 10:29:31 2021 +0800

    Rebuilt ElasticSearch client on top of their REST API
    Remove page path from browser log query condition
    Add Python Falcon Plugin Logo (#528)
---
 oap-server-bom/pom.xml                             |   8 +-
 .../server-bootstrap/src/main/resources/log4j2.xml |   1 +
 .../storage/type/StorageDataComplexObject.java     |  23 ++
 oap-server/server-library/library-client/pom.xml   |   6 +
 .../client/elasticsearch/ElasticSearchClient.java  | 407 ++++++++-------------
 ...InsertRequest.java => IndexRequestWrapper.java} |  25 +-
 .../client/elasticsearch/UpdateRequestWrapper.java |  38 ++
 .../elasticsearch/ITElasticSearchClient.java       |  52 +--
 .../{ => library-elasticsearch-client}/pom.xml     |  33 +-
 .../library/elasticsearch/ElasticSearchClient.java | 148 ++++++++
 .../elasticsearch/ElasticSearchClientBuilder.java  | 150 ++++++++
 .../elasticsearch/ElasticSearchVersion.java        |  80 ++++
 .../library/elasticsearch/bulk/BulkProcessor.java  | 130 +++++++
 .../elasticsearch/bulk/BulkProcessorBuilder.java   |  63 ++++
 .../library/elasticsearch/client/AliasClient.java  |  64 ++++
 .../elasticsearch/client/DocumentClient.java       | 129 +++++++
 .../library/elasticsearch/client/IndexClient.java  | 106 ++++++
 .../library/elasticsearch/client/SearchClient.java |  63 ++++
 .../elasticsearch/client/TemplateClient.java       | 106 ++++++
 .../elasticsearch/requests/IndexRequest.java}      |  32 +-
 .../elasticsearch/requests/UpdateRequest.java}     |  32 +-
 .../requests/factory/AliasFactory.java}            |  22 +-
 .../requests/factory/BulkFactory.java}             |  23 +-
 .../elasticsearch/requests/factory/Codec.java}     |  25 +-
 .../requests/factory/DocumentFactory.java          |  56 +++
 .../requests/factory/IndexFactory.java}            |  34 +-
 .../requests/factory/RequestFactory.java           |  91 +++++
 .../requests/factory/SearchFactory.java}           |  23 +-
 .../requests/factory/TemplateFactory.java}         |  30 +-
 .../requests/factory/v6/V6AliasFactory.java}       |  25 +-
 .../requests/factory/v6/V6BulkFactory.java}        |  27 +-
 .../requests/factory/v6/V6DocumentFactory.java     | 160 ++++++++
 .../requests/factory/v6/V6IndexFactory.java        |  91 +++++
 .../requests/factory/v6/V6RequestFactory.java      |  67 ++++
 .../requests/factory/v6/V6SearchFactory.java       |  48 +++
 .../requests/factory/v6/V6TemplateFactory.java     |  79 ++++
 .../factory/v6/codec/BoolQuerySerializer.java      |  68 ++++
 .../factory/v6/codec/IndexRequestSerializer.java   |  52 +++
 .../factory/v6/codec/RangeQuerySerializer.java     |  62 ++++
 .../requests/factory/v6/codec/TermSerializer.java  |  44 +++
 .../factory/v6/codec/UpdateRequestSerializer.java  |  58 +++
 .../requests/factory/v6/codec/V6Codec.java         |  82 +++++
 .../elasticsearch/requests/search/BoolQuery.java}  |  31 +-
 .../requests/search/BoolQueryBuilder.java          | 133 +++++++
 .../elasticsearch/requests/search/IdsQuery.java    |  53 +++
 .../requests/search/MatchPhaseQuery.java           |  53 +++
 .../elasticsearch/requests/search/MatchQuery.java  |  53 +++
 .../elasticsearch/requests/search/Query.java       |  80 ++++
 .../requests/search/QueryBuilder.java}             |  23 +-
 .../elasticsearch/requests/search/RangeQuery.java} |  32 +-
 .../requests/search/RangeQueryBuilder.java         |  70 ++++
 .../elasticsearch/requests/search/Search.java}     |  38 +-
 .../requests/search/SearchBuilder.java             | 122 ++++++
 .../elasticsearch/requests/search/Sort.java}       |  32 +-
 .../elasticsearch/requests/search/Sorts.java       |  74 ++++
 .../elasticsearch/requests/search/TermQuery.java}  |  28 +-
 .../elasticsearch/requests/search/TermsQuery.java} |  29 +-
 .../requests/search/aggregation/Aggregation.java   |  50 +++
 .../search/aggregation/AggregationBuilder.java}    |  23 +-
 .../search/aggregation/AvgAggregation.java         |  61 +++
 .../search/aggregation/AvgAggregationBuilder.java} |  28 +-
 .../requests/search/aggregation/BucketOrder.java}  |  30 +-
 .../search/aggregation/MaxAggregation.java}        |  22 +-
 .../search/aggregation/MaxAggregationBuilder.java} |  28 +-
 .../search/aggregation/MinAggregation.java}        |  31 +-
 .../search/aggregation/MinAggregationBuilder.java} |  28 +-
 .../search/aggregation/SumAggregation.java}        |  31 +-
 .../search/aggregation/SumAggregationBuilder.java} |  28 +-
 .../search/aggregation/TermsAggregation.java       |  69 ++++
 .../aggregation/TermsAggregationBuilder.java       |  87 +++++
 .../library/elasticsearch/response/Document.java}  |  30 +-
 .../library/elasticsearch/response/Documents.java} |  29 +-
 .../library/elasticsearch/response/NodeInfo.java}  |  27 +-
 .../elasticsearch/response/search/SearchHit.java}  |  37 +-
 .../elasticsearch/response/search/SearchHits.java} |  33 +-
 .../response/search/SearchResponse.java}           |  28 +-
 .../elasticsearch/ITElasticSearchClientTest.java   | 106 ++++++
 oap-server/server-library/pom.xml                  |   3 +-
 .../StorageModuleElasticsearchProvider.java        |   1 +
 .../elasticsearch/base/BatchProcessEsDAO.java      |  12 +-
 .../storage/plugin/elasticsearch/base/EsDAO.java   |   1 +
 .../elasticsearch/base/HistoryDeleteEsDAO.java     |   5 +-
 .../plugin/elasticsearch/base/ManagementEsDAO.java |  16 +-
 .../plugin/elasticsearch/base/MetricsEsDAO.java    |  23 +-
 .../plugin/elasticsearch/base/NoneStreamEsDAO.java |   7 +-
 .../plugin/elasticsearch/base/RecordEsDAO.java     |   6 +-
 .../cache/NetworkAddressAliasEsDAO.java            |  24 +-
 .../elasticsearch/query/AggregationQueryEsDAO.java | 115 +++---
 .../elasticsearch/query/AlarmQueryEsDAO.java       |  50 +--
 .../elasticsearch/query/ESEventQueryDAO.java       | 139 +++----
 .../plugin/elasticsearch/query/LogQueryEsDAO.java  |  90 ++---
 .../elasticsearch/query/MetadataQueryEsDAO.java    | 165 ++++-----
 .../elasticsearch/query/MetricsQueryEsDAO.java     | 191 +++++-----
 .../elasticsearch/query/ProfileTaskLogEsDAO.java   |  40 +-
 .../elasticsearch/query/ProfileTaskQueryEsDAO.java |  89 ++---
 .../query/ProfileThreadSnapshotQueryEsDAO.java     | 185 +++++-----
 .../elasticsearch/query/TopNRecordsQueryEsDAO.java |  50 ++-
 .../elasticsearch/query/TopologyQueryEsDAO.java    | 291 ++++++++-------
 .../elasticsearch/query/TraceQueryEsDAO.java       |  94 ++---
 .../query/UITemplateManagementEsDAO.java           |  82 +++--
 .../client/ElasticSearch7Client.java               |  42 +--
 101 files changed, 4603 insertions(+), 1638 deletions(-)

diff --git a/oap-server-bom/pom.xml b/oap-server-bom/pom.xml
index 5b4e179..4f46c52 100644
--- a/oap-server-bom/pom.xml
+++ b/oap-server-bom/pom.xml
@@ -72,7 +72,7 @@
         <flatbuffers-java.version>1.12.0</flatbuffers-java.version>
         <postgresql.version>42.2.18</postgresql.version>
         <jetcd.version>0.5.3</jetcd.version>
-        <testcontainers.version>1.15.3</testcontainers.version>
+        <testcontainers.version>1.16.0</testcontainers.version>
     </properties>
 
     <dependencyManagement>
@@ -497,6 +497,12 @@
                 <version>${testcontainers.version}</version>
                 <scope>test</scope>
             </dependency>
+            <dependency>
+                <groupId>org.testcontainers</groupId>
+                <artifactId>elasticsearch</artifactId>
+                <version>${testcontainers.version}</version>
+                <scope>test</scope>
+            </dependency>
         </dependencies>
     </dependencyManagement>
 </project>
diff --git a/oap-server/server-bootstrap/src/main/resources/log4j2.xml b/oap-server/server-bootstrap/src/main/resources/log4j2.xml
index 92c8884..8fade37 100644
--- a/oap-server/server-bootstrap/src/main/resources/log4j2.xml
+++ b/oap-server/server-bootstrap/src/main/resources/log4j2.xml
@@ -39,6 +39,7 @@
         <logger name="org.apache.skywalking.oap.server.library.buffer" level="INFO"/>
         <logger name="org.apache.skywalking.oap.server.receiver.envoy.MetricServiceGRPCHandler" level="INFO"/>
         <logger name="org.apache.skywalking.oap.meter.analyzer.prometheus.PrometheusMetricConverter" level="INFO"/>
+        <logger name="org.elasticsearch.client" level="TRACE"/>
         <Root level="DEBUG">
             <AppenderRef ref="Console"/>
         </Root>
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
index ebad19e..980e878 100644
--- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
+++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
@@ -18,9 +18,16 @@
 
 package org.apache.skywalking.oap.server.core.storage.type;
 
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import com.fasterxml.jackson.databind.annotation.JsonSerialize;
+import com.fasterxml.jackson.databind.ser.std.StdSerializer;
+import java.io.IOException;
+
 /**
  * StorageDataComplexObject implementation supports String-Object interconversion.
  */
+@JsonSerialize(using = StorageDataComplexObject.Serializer.class)
 public interface StorageDataComplexObject<T> {
     /**
      * @return string representing this object.
@@ -36,4 +43,20 @@ public interface StorageDataComplexObject<T> {
      * Initialize the object based on the given source.
      */
     void copyFrom(T source);
+
+    @SuppressWarnings("rawtypes")
+    class Serializer extends StdSerializer<StorageDataComplexObject> {
+        protected Serializer() {
+            super(StorageDataComplexObject.class);
+        }
+
+        @Override
+        public void serialize(
+            final StorageDataComplexObject value,
+            final JsonGenerator gen,
+            final SerializerProvider provider)
+            throws IOException {
+            gen.writeString(value.toStorageData());
+        }
+    }
 }
diff --git a/oap-server/server-library/library-client/pom.xml b/oap-server/server-library/library-client/pom.xml
index d5c6b82..43c4e56 100755
--- a/oap-server/server-library/library-client/pom.xml
+++ b/oap-server/server-library/library-client/pom.xml
@@ -40,6 +40,12 @@
         </dependency>
 
         <dependency>
+            <groupId>org.apache.skywalking</groupId>
+            <artifactId>library-elasticsearch-client</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
             <groupId>io.grpc</groupId>
             <artifactId>grpc-core</artifactId>
         </dependency>
diff --git a/oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ElasticSearchClient.java b/oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ElasticSearchClient.java
index 38e3af6..44be01e 100644
--- a/oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ElasticSearchClient.java
+++ b/oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ElasticSearchClient.java
@@ -19,13 +19,9 @@
 package org.apache.skywalking.oap.server.library.client.elasticsearch;
 
 import com.google.common.base.Splitter;
-import com.google.gson.Gson;
-import com.google.gson.JsonObject;
-import com.google.gson.reflect.TypeToken;
+import com.google.common.collect.ImmutableMap;
 import java.io.IOException;
 import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.lang.reflect.Type;
 import java.nio.file.Files;
 import java.nio.file.Paths;
 import java.security.KeyManagementException;
@@ -33,8 +29,9 @@ import java.security.KeyStore;
 import java.security.KeyStoreException;
 import java.security.NoSuchAlgorithmException;
 import java.security.cert.CertificateException;
-import java.util.ArrayList;
+import java.time.Duration;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.LinkedList;
@@ -46,57 +43,30 @@ import javax.net.ssl.SSLContext;
 import lombok.RequiredArgsConstructor;
 import lombok.Setter;
 import lombok.extern.slf4j.Slf4j;
-import org.apache.http.HttpEntity;
 import org.apache.http.HttpHost;
-import org.apache.http.HttpStatus;
 import org.apache.http.auth.AuthScope;
 import org.apache.http.auth.UsernamePasswordCredentials;
 import org.apache.http.client.CredentialsProvider;
-import org.apache.http.client.methods.HttpDelete;
-import org.apache.http.client.methods.HttpGet;
-import org.apache.http.client.methods.HttpHead;
-import org.apache.http.client.methods.HttpPost;
-import org.apache.http.client.methods.HttpPut;
-import org.apache.http.entity.ContentType;
 import org.apache.http.impl.client.BasicCredentialsProvider;
-import org.apache.http.nio.entity.NStringEntity;
 import org.apache.http.ssl.SSLContextBuilder;
 import org.apache.http.ssl.SSLContexts;
 import org.apache.skywalking.apm.util.StringUtil;
+import org.apache.skywalking.library.elasticsearch.requests.search.Search;
+import org.apache.skywalking.library.elasticsearch.response.Document;
+import org.apache.skywalking.library.elasticsearch.response.Documents;
 import org.apache.skywalking.oap.server.library.client.Client;
 import org.apache.skywalking.oap.server.library.client.healthcheck.DelegatedHealthChecker;
 import org.apache.skywalking.oap.server.library.client.healthcheck.HealthCheckable;
-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.library.util.HealthChecker;
-import org.elasticsearch.action.admin.indices.create.CreateIndexRequest;
-import org.elasticsearch.action.admin.indices.create.CreateIndexResponse;
-import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest;
-import org.elasticsearch.action.admin.indices.delete.DeleteIndexResponse;
-import org.elasticsearch.action.admin.indices.get.GetIndexRequest;
-import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest;
-import org.elasticsearch.action.admin.indices.mapping.put.PutMappingResponse;
-import org.elasticsearch.action.bulk.BackoffPolicy;
 import org.elasticsearch.action.bulk.BulkProcessor;
 import org.elasticsearch.action.bulk.BulkRequest;
 import org.elasticsearch.action.bulk.BulkResponse;
-import org.elasticsearch.action.get.GetRequest;
-import org.elasticsearch.action.get.GetResponse;
-import org.elasticsearch.action.index.IndexRequest;
 import org.elasticsearch.action.search.SearchRequest;
 import org.elasticsearch.action.search.SearchResponse;
-import org.elasticsearch.action.support.ActiveShardCount;
 import org.elasticsearch.action.support.IndicesOptions;
-import org.elasticsearch.action.support.WriteRequest;
-import org.elasticsearch.client.Response;
-import org.elasticsearch.client.ResponseException;
 import org.elasticsearch.client.RestClient;
 import org.elasticsearch.client.RestClientBuilder;
 import org.elasticsearch.client.RestHighLevelClient;
-import org.elasticsearch.common.unit.TimeValue;
-import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.common.xcontent.XContentType;
-import org.elasticsearch.index.query.QueryBuilders;
 import org.elasticsearch.search.builder.SearchSourceBuilder;
 
 /**
@@ -106,22 +76,36 @@ import org.elasticsearch.search.builder.SearchSourceBuilder;
 @RequiredArgsConstructor
 public class ElasticSearchClient implements Client, HealthCheckable {
     public static final String TYPE = "type";
+
     protected final String clusterNodes;
+
     protected final String protocol;
+
     private final String trustStorePath;
+
     @Setter
     private volatile String trustStorePass;
+
     @Setter
     private volatile String user;
+
     @Setter
     private volatile String password;
+
     private final List<IndexNameConverter> indexNameConverters;
+
     protected volatile RestHighLevelClient client;
+
     protected DelegatedHealthChecker healthChecker = new DelegatedHealthChecker();
+
     protected final ReentrantLock connectLock = new ReentrantLock();
+
     private final int connectTimeout;
+
     private final int socketTimeout;
 
+    org.apache.skywalking.library.elasticsearch.ElasticSearchClient esClient;
+
     public ElasticSearchClient(String clusterNodes,
                                String protocol,
                                String trustStorePath,
@@ -140,10 +124,22 @@ public class ElasticSearchClient implements Client, HealthCheckable {
         this.trustStorePass = trustStorePass;
         this.connectTimeout = connectTimeout;
         this.socketTimeout = socketTimeout;
+        this.esClient = org.apache.skywalking.library.elasticsearch.ElasticSearchClient
+            .builder()
+            .endpoints(clusterNodes.split(","))
+            .protocol(protocol)
+            .trustStorePath(trustStorePath)
+            .trustStorePass(trustStorePass)
+            .username(user)
+            .password(password)
+            .connectTimeout(connectTimeout)
+            .build();
     }
 
     @Override
-    public void connect() throws IOException, KeyStoreException, NoSuchAlgorithmException, KeyManagementException, CertificateException {
+    public void connect()
+        throws IOException, KeyStoreException, NoSuchAlgorithmException, KeyManagementException,
+        CertificateException {
         connectLock.lock();
         try {
             List<HttpHost> hosts = parseClusterNodes(protocol, clusterNodes);
@@ -156,17 +152,21 @@ public class ElasticSearchClient implements Client, HealthCheckable {
             }
             client = createClient(hosts);
             client.ping();
+            esClient.connect();
         } finally {
             connectLock.unlock();
         }
     }
 
     protected RestHighLevelClient createClient(
-        final List<HttpHost> pairsList) throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException, KeyManagementException {
+        final List<HttpHost> pairsList)
+        throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException,
+        KeyManagementException {
         RestClientBuilder builder;
         if (StringUtil.isNotEmpty(user) && StringUtil.isNotEmpty(password)) {
             final CredentialsProvider credentialsProvider = new BasicCredentialsProvider();
-            credentialsProvider.setCredentials(AuthScope.ANY, new UsernamePasswordCredentials(user, password));
+            credentialsProvider.setCredentials(
+                AuthScope.ANY, new UsernamePasswordCredentials(user, password));
 
             if (StringUtil.isEmpty(trustStorePath)) {
                 builder = RestClient.builder(pairsList.toArray(new HttpHost[0]))
@@ -178,13 +178,15 @@ public class ElasticSearchClient implements Client, HealthCheckable {
                 try (InputStream is = Files.newInputStream(Paths.get(trustStorePath))) {
                     truststore.load(is, trustStorePass.toCharArray());
                 }
-                SSLContextBuilder sslBuilder = SSLContexts.custom().loadTrustMaterial(truststore, null);
+                SSLContextBuilder sslBuilder =
+                    SSLContexts.custom().loadTrustMaterial(truststore, null);
                 final SSLContext sslContext = sslBuilder.build();
                 builder = RestClient.builder(pairsList.toArray(new HttpHost[0]))
                                     .setHttpClientConfigCallback(
                                         httpClientBuilder -> httpClientBuilder.setDefaultCredentialsProvider(
-                                            credentialsProvider)
-                                                                              .setSSLContext(sslContext));
+                                                                                  credentialsProvider)
+                                                                              .setSSLContext(
+                                                                                  sslContext));
             }
         } else {
             builder = RestClient.builder(pairsList.toArray(new HttpHost[0]));
@@ -201,6 +203,7 @@ public class ElasticSearchClient implements Client, HealthCheckable {
     @Override
     public void shutdown() throws IOException {
         client.close();
+        esClient.close();
     }
 
     public static List<HttpHost> parseClusterNodes(String protocol, String nodes) {
@@ -220,22 +223,14 @@ public class ElasticSearchClient implements Client, HealthCheckable {
     public boolean createIndex(String indexName) throws IOException {
         indexName = formatIndexName(indexName);
 
-        CreateIndexRequest request = new CreateIndexRequest(indexName);
-        CreateIndexResponse response = client.indices().create(request);
-        log.debug("create {} index finished, isAcknowledged: {}", indexName, response.isAcknowledged());
-        return response.isAcknowledged();
+        return esClient.index().create(indexName);
     }
 
-    public boolean updateIndexMapping(String indexName, Map<String, Object> mapping) throws IOException {
+    public boolean updateIndexMapping(String indexName, Map<String, Object> mapping)
+        throws IOException {
         indexName = formatIndexName(indexName);
-        Map<String, Object> properties = (Map<String, Object>) mapping.get(ElasticSearchClient.TYPE);
-        PutMappingRequest putMappingRequest = new PutMappingRequest(indexName);
-        Gson gson = new Gson();
-        putMappingRequest.type(ElasticSearchClient.TYPE);
-        putMappingRequest.source(gson.toJson(properties), XContentType.JSON);
-        PutMappingResponse response = client.indices().putMapping(putMappingRequest);
-        log.debug("put {} index mapping finished, isAcknowledged: {}", indexName, response.isAcknowledged());
-        return response.isAcknowledged();
+
+        return esClient.index().putMapping(indexName, TYPE, mapping);
     }
 
     public Map<String, Object> getIndex(String indexName) throws IOException {
@@ -244,75 +239,29 @@ public class ElasticSearchClient implements Client, HealthCheckable {
         }
         indexName = formatIndexName(indexName);
         try {
-            Response response = client.getLowLevelClient()
-                                      .performRequest(HttpGet.METHOD_NAME, "/" + indexName);
-            int statusCode = response.getStatusLine().getStatusCode();
-            if (statusCode != HttpStatus.SC_OK) {
-                healthChecker.health();
-                throw new IOException(
-                    "The response status code of template exists request should be 200, but it is " + statusCode);
+            final Map<String, Object> indices = esClient.index().get(indexName);
+            if (indices.containsKey(indexName)) {
+                // noinspection unchecked
+                return (Map<String, Object>) indices.get(indexName);
             }
-            Type type = new TypeToken<HashMap<String, Object>>() {
-            }.getType();
-            Map<String, Object> templates = new Gson().<HashMap<String, Object>>fromJson(
-                new InputStreamReader(response.getEntity().getContent()),
-                type
-            );
-            return (Map<String, Object>) Optional.ofNullable(templates.get(indexName)).orElse(new HashMap<>());
-        } catch (ResponseException e) {
-            if (e.getResponse().getStatusLine().getStatusCode() == HttpStatus.SC_NOT_FOUND) {
-                return new HashMap<>();
-            }
-            healthChecker.unHealth(e);
-            throw e;
-        } catch (IOException t) {
+            return Collections.emptyMap();
+        } catch (Exception t) {
             healthChecker.unHealth(t);
             throw t;
         }
     }
 
-    public boolean createIndex(String indexName, Map<String, Object> settings,
-                               Map<String, Object> mapping) throws IOException {
-        indexName = formatIndexName(indexName);
-        CreateIndexRequest request = new CreateIndexRequest(indexName);
-        Gson gson = new Gson();
-        request.settings(gson.toJson(settings), XContentType.JSON);
-        request.mapping(TYPE, gson.toJson(mapping), XContentType.JSON);
-        CreateIndexResponse response = client.indices().create(request);
-        log.debug("create {} index finished, isAcknowledged: {}", indexName, response.isAcknowledged());
-        return response.isAcknowledged();
-    }
-
-    public List<String> retrievalIndexByAliases(String aliases) throws IOException {
+    public Collection<String> retrievalIndexByAliases(String aliases) throws IOException {
         aliases = formatIndexName(aliases);
-        Response response;
-        try {
-            response = client.getLowLevelClient().performRequest(HttpGet.METHOD_NAME, "/_alias/" + aliases);
-            healthChecker.health();
-        } catch (Throwable t) {
-            healthChecker.unHealth(t);
-            throw t;
-        }
-        if (HttpStatus.SC_OK == response.getStatusLine().getStatusCode()) {
-            Gson gson = new Gson();
-            InputStreamReader reader;
-            try {
-                reader = new InputStreamReader(response.getEntity().getContent());
-            } catch (Throwable t) {
-                healthChecker.unHealth(t);
-                throw t;
-            }
-            JsonObject responseJson = gson.fromJson(reader, JsonObject.class);
-            log.debug("retrieval indexes by aliases {}, response is {}", aliases, responseJson);
-            return new ArrayList<>(responseJson.keySet());
-        }
-        return Collections.emptyList();
+
+        return esClient.alias().indices(aliases).keySet();
     }
 
     /**
-     * If your indexName is retrieved from elasticsearch through {@link #retrievalIndexByAliases(String)} or some other
-     * method and it already contains namespace. Then you should delete the index by this method, this method will no
-     * longer concatenate namespace.
+     * If your indexName is retrieved from elasticsearch through {@link
+     * #retrievalIndexByAliases(String)} or some other method and it already contains namespace.
+     * Then you should delete the index by this method, this method will no longer concatenate
+     * namespace.
      * <p>
      * https://github.com/apache/skywalking/pull/3017
      */
@@ -321,8 +270,8 @@ public class ElasticSearchClient implements Client, HealthCheckable {
     }
 
     /**
-     * If your indexName is obtained from metadata or configuration and without namespace. Then you should delete the
-     * index by this method, this method automatically concatenates namespace.
+     * If your indexName is obtained from metadata or configuration and without namespace. Then you
+     * should delete the index by this method, this method automatically concatenates namespace.
      * <p>
      * https://github.com/apache/skywalking/pull/3017
      */
@@ -334,111 +283,72 @@ public class ElasticSearchClient implements Client, HealthCheckable {
         if (formatIndexName) {
             indexName = formatIndexName(indexName);
         }
-        DeleteIndexRequest request = new DeleteIndexRequest(indexName);
-        DeleteIndexResponse response;
-        response = client.indices().delete(request);
-        log.debug("delete {} index finished, isAcknowledged: {}", indexName, response.isAcknowledged());
-        return response.isAcknowledged();
+        return esClient.index().delete(indexName);
     }
 
     public boolean isExistsIndex(String indexName) throws IOException {
         indexName = formatIndexName(indexName);
-        GetIndexRequest request = new GetIndexRequest();
-        request.indices(indexName);
-        return client.indices().exists(request);
+
+        return esClient.index().exists(indexName);
     }
 
     public Map<String, Object> getTemplate(String name) throws IOException {
         name = formatIndexName(name);
+
         try {
-            Response response = client.getLowLevelClient()
-                                      .performRequest(HttpGet.METHOD_NAME, "/_template/" + name);
-            int statusCode = response.getStatusLine().getStatusCode();
-            if (statusCode != HttpStatus.SC_OK) {
-                healthChecker.health();
-                throw new IOException(
-                    "The response status code of template exists request should be 200, but it is " + statusCode);
-            }
-            Type type = new TypeToken<HashMap<String, Object>>() {
-            }.getType();
-            Map<String, Object> templates = new Gson().<HashMap<String, Object>>fromJson(
-                new InputStreamReader(response.getEntity().getContent()),
-                type
-            );
+            Map<String, Object> templates = esClient.templates().get(name);
             if (templates.containsKey(name)) {
+                // noinspection unchecked
                 return (Map<String, Object>) templates.get(name);
             }
-            return new HashMap<>();
-        } catch (ResponseException e) {
-            if (e.getResponse().getStatusLine().getStatusCode() == HttpStatus.SC_NOT_FOUND) {
-                return new HashMap<>();
-            }
+            return Collections.emptyMap();
+        } catch (Exception e) {
             healthChecker.unHealth(e);
             throw e;
-        } catch (IOException t) {
-            healthChecker.unHealth(t);
-            throw t;
         }
     }
 
     public boolean isExistsTemplate(String indexName) throws IOException {
         indexName = formatIndexName(indexName);
 
-        Response response = client.getLowLevelClient().performRequest(HttpHead.METHOD_NAME, "/_template/" + indexName);
-
-        int statusCode = response.getStatusLine().getStatusCode();
-        if (statusCode == HttpStatus.SC_OK) {
-            return true;
-        } else if (statusCode == HttpStatus.SC_NOT_FOUND) {
-            return false;
-        } else {
-            throw new IOException(
-                "The response status code of template exists request should be 200 or 404, but it is " + statusCode);
-        }
+        return esClient.templates().exists(indexName);
     }
 
     public boolean createOrUpdateTemplate(String indexName, Map<String, Object> settings,
-                                          Map<String, Object> mapping, int order) throws IOException {
+                                          Map<String, Object> mapping, int order)
+        throws IOException {
         indexName = formatIndexName(indexName);
 
-        String[] patterns = new String[] {indexName + "-*"};
-
-        Map<String, Object> aliases = new HashMap<>();
-        aliases.put(indexName, new JsonObject());
-
-        Map<String, Object> template = new HashMap<>();
-        template.put("index_patterns", patterns);
-        template.put("aliases", aliases);
-        template.put("settings", settings);
-        template.put("mappings", mapping);
-        template.put("order", order);
-
-        HttpEntity entity = new NStringEntity(new Gson().toJson(template), ContentType.APPLICATION_JSON);
-
-        Response response = client.getLowLevelClient()
-                                  .performRequest(
-                                      HttpPut.METHOD_NAME, "/_template/" + indexName, Collections.emptyMap(), entity);
-        return response.getStatusLine().getStatusCode() == HttpStatus.SC_OK;
+        return esClient.templates().createOrUpdate(indexName, settings, mapping, order);
     }
 
     public boolean deleteTemplate(String indexName) throws IOException {
         indexName = formatIndexName(indexName);
-        Response response = client.getLowLevelClient()
-                                  .performRequest(HttpDelete.METHOD_NAME, "/_template/" + indexName);
-        return response.getStatusLine().getStatusCode() == HttpStatus.SC_OK;
+
+        return esClient.templates().delete(indexName);
     }
 
     public SearchResponse search(IndexNameMaker indexNameMaker,
                                  SearchSourceBuilder searchSourceBuilder) throws IOException {
-        String[] indexNames = Arrays.stream(indexNameMaker.make()).map(this::formatIndexName).toArray(String[]::new);
+        String[] indexNames =
+            Arrays.stream(indexNameMaker.make()).map(this::formatIndexName).toArray(String[]::new);
         return doSearch(searchSourceBuilder, indexNames);
     }
 
-    public SearchResponse search(String indexName, SearchSourceBuilder searchSourceBuilder) throws IOException {
+    public SearchResponse search(String indexName, SearchSourceBuilder searchSourceBuilder)
+        throws IOException {
         indexName = formatIndexName(indexName);
         return doSearch(searchSourceBuilder, indexName);
     }
 
+    // TODO
+    public org.apache.skywalking.library.elasticsearch.response.search.SearchResponse search(
+        String indexName, Search search)
+        throws IOException {
+        indexName = formatIndexName(indexName);
+        return esClient.search(search, indexName);
+    }
+
     protected SearchResponse doSearch(SearchSourceBuilder searchSourceBuilder,
                                       String... indexNames) throws IOException {
         SearchRequest searchRequest = new SearchRequest(indexNames);
@@ -472,40 +382,34 @@ public class ElasticSearchClient implements Client, HealthCheckable {
         }
     }
 
-    public GetResponse get(String indexName, String id) throws IOException {
+    public Optional<Document> get(String indexName, String id) throws IOException {
         indexName = formatIndexName(indexName);
-        GetRequest request = new GetRequest(indexName, TYPE, id);
-        try {
-            GetResponse response = client.get(request);
-            healthChecker.health();
-            return response;
-        } catch (Throwable t) {
-            healthChecker.unHealth(t);
-            throw t;
-        }
+
+        return esClient.documents().get(indexName, TYPE, id);
     }
 
-    public SearchResponse ids(String indexName, String[] ids) throws IOException {
+    public boolean existDoc(String indexName, String id) throws IOException {
         indexName = formatIndexName(indexName);
 
-        SearchRequest searchRequest = new SearchRequest(indexName);
-        searchRequest.types(TYPE);
-        searchRequest.source().query(QueryBuilders.idsQuery().addIds(ids)).size(ids.length);
-        try {
-            SearchResponse response = client.search(searchRequest);
-            healthChecker.health();
-            return response;
-        } catch (Throwable t) {
-            healthChecker.unHealth(t);
-            throw t;
-        }
+        return esClient.documents().exists(indexName, TYPE, id);
+    }
+
+    public Optional<Documents> ids(String indexName, Iterable<String> ids) {
+        indexName = formatIndexName(indexName);
+
+        return esClient.documents().mget(indexName, TYPE, ids);
     }
 
-    public void forceInsert(String indexName, String id, XContentBuilder source) throws IOException {
-        IndexRequest request = (IndexRequest) prepareInsert(indexName, id, source);
-        request.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE);
+    public Optional<Documents> ids(String indexName, String[] ids) {
+        return ids(indexName, Arrays.asList(ids));
+    }
+
+    public void forceInsert(String indexName, String id, Map<String, Object> source)
+        throws IOException {
+        IndexRequestWrapper wrapper = prepareInsert(indexName, id, source);
+        Map<String, Object> params = ImmutableMap.of("refresh", "true");
         try {
-            client.index(request);
+            esClient.documents().index(wrapper.getRequest(), params);
             healthChecker.health();
         } catch (Throwable t) {
             healthChecker.unHealth(t);
@@ -513,12 +417,12 @@ public class ElasticSearchClient implements Client, HealthCheckable {
         }
     }
 
-    public void forceUpdate(String indexName, String id, XContentBuilder source) throws IOException {
-        org.elasticsearch.action.update.UpdateRequest request = (org.elasticsearch.action.update.UpdateRequest) prepareUpdate(
-            indexName, id, source);
-        request.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE);
+    public void forceUpdate(String indexName, String id, Map<String, Object> source)
+        throws IOException {
+        UpdateRequestWrapper wrapper = prepareUpdate(indexName, id, source);
+        Map<String, Object> params = ImmutableMap.of("refresh", "true");
         try {
-            client.update(request);
+            esClient.documents().update(wrapper.getRequest(), params);
             healthChecker.health();
         } catch (Throwable t) {
             healthChecker.unHealth(t);
@@ -526,55 +430,46 @@ public class ElasticSearchClient implements Client, HealthCheckable {
         }
     }
 
-    public InsertRequest prepareInsert(String indexName, String id, XContentBuilder source) {
+    public IndexRequestWrapper prepareInsert(String indexName, String id,
+                                             Map<String, Object> source) {
         indexName = formatIndexName(indexName);
-        return new ElasticSearchInsertRequest(indexName, TYPE, id).source(source);
+        return new IndexRequestWrapper(indexName, TYPE, id, source);
     }
 
-    public UpdateRequest prepareUpdate(String indexName, String id, XContentBuilder source) {
+    public UpdateRequestWrapper prepareUpdate(String indexName, String id,
+                                              Map<String, Object> source) {
         indexName = formatIndexName(indexName);
-        return new ElasticSearchUpdateRequest(indexName, TYPE, id).doc(source);
+        return new UpdateRequestWrapper(indexName, TYPE, id, source);
     }
 
-    public int delete(String indexName, String timeBucketColumnName, long endTimeBucket) throws IOException {
+    public int delete(String indexName, String timeBucketColumnName, long endTimeBucket)
+        throws IOException {
         indexName = formatIndexName(indexName);
-        Map<String, String> params = Collections.singletonMap("conflicts", "proceed");
-        String jsonString = "{" + "  \"query\": {" + "    \"range\": {" + "      \"" + timeBucketColumnName + "\": {" + "        \"lte\": " + endTimeBucket + "      }" + "    }" + "  }" + "}";
-        HttpEntity entity = new NStringEntity(jsonString, ContentType.APPLICATION_JSON);
-        Response response = client.getLowLevelClient()
-                                  .performRequest(
-                                      HttpPost.METHOD_NAME, "/" + indexName + "/_delete_by_query", params, entity);
-        log.debug("delete indexName: {}, jsonString : {}", indexName, jsonString);
-        return response.getStatusLine().getStatusCode();
-    }
+        Map<String, Object> params = Collections.singletonMap("conflicts", "proceed");
+        String query = "" +
+            "{" +
+            "  \"query\": {" +
+            "    \"range\": {" +
+            "      \"" + timeBucketColumnName + "\": {" +
+            "        \"lte\": " + endTimeBucket +
+            "      }" +
+            "    }" +
+            "  }" +
+            "}";
 
-    /**
-     * @since 8.7.0 SkyWalking don't use sync bulk anymore. This method is just kept for unexpected case in the future.
-     */
-    @Deprecated
-    public void synchronousBulk(BulkRequest request) {
-        request.timeout(TimeValue.timeValueMinutes(2));
-        request.setRefreshPolicy(WriteRequest.RefreshPolicy.WAIT_UNTIL);
-        request.waitForActiveShards(ActiveShardCount.ONE);
-        try {
-            int size = request.requests().size();
-            BulkResponse responses = client.bulk(request);
-            log.info("Synchronous bulk took time: {} millis, size: {}", responses.getTook().getMillis(), size);
-            healthChecker.health();
-        } catch (Throwable t) {
-            healthChecker.unHealth(t);
-        }
-    }
+        esClient.documents().delete(indexName, TYPE, query, params);
 
-    public BulkProcessor createBulkProcessor(int bulkActions, int flushInterval, int concurrentRequests) {
-        BulkProcessor.Listener listener = createBulkListener();
+        return 0; // TODO
+    }
 
-        return BulkProcessor.builder(client::bulkAsync, listener)
-                            .setBulkActions(bulkActions)
-                            .setFlushInterval(TimeValue.timeValueSeconds(flushInterval))
-                            .setConcurrentRequests(concurrentRequests)
-                            .setBackoffPolicy(BackoffPolicy.exponentialBackoff(TimeValue.timeValueMillis(100), 3))
-                            .build();
+    public org.apache.skywalking.library.elasticsearch.bulk.BulkProcessor createBulkProcessor(
+        int bulkActions, int flushInterval,
+        int concurrentRequests) {
+        return esClient.bulkProcessor()
+                       .bulkActions(bulkActions)
+                       .flushInterval(Duration.ofSeconds(flushInterval))
+                       .concurrentRequests(concurrentRequests)
+                       .build();
     }
 
     protected BulkProcessor.Listener createBulkListener() {
@@ -588,11 +483,15 @@ public class ElasticSearchClient implements Client, HealthCheckable {
             @Override
             public void afterBulk(long executionId, BulkRequest request, BulkResponse response) {
                 if (response.hasFailures()) {
-                    log.warn("Bulk [{}] executed with failures:[{}]", executionId, response.buildFailureMessage());
+                    log.warn(
+                        "Bulk [{}] executed with failures:[{}]", executionId,
+                        response.buildFailureMessage()
+                    );
                 } else {
                     log.info(
-                        "Bulk execution id [{}] completed in {} milliseconds, size: {}", executionId, response.getTook()
-                                                                                                              .getMillis(),
+                        "Bulk execution id [{}] completed in {} milliseconds, size: {}",
+                        executionId, response.getTook()
+                                             .getMillis(),
                         request
                             .requests()
                             .size()
diff --git a/oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ElasticSearchInsertRequest.java b/oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/IndexRequestWrapper.java
similarity index 61%
copy from oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ElasticSearchInsertRequest.java
copy to oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/IndexRequestWrapper.java
index 9c0655c..b8ee658 100644
--- a/oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ElasticSearchInsertRequest.java
+++ b/oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/IndexRequestWrapper.java
@@ -17,19 +17,22 @@
 
 package org.apache.skywalking.oap.server.library.client.elasticsearch;
 
+import java.util.Map;
+import lombok.Getter;
+import org.apache.skywalking.library.elasticsearch.requests.IndexRequest;
 import org.apache.skywalking.oap.server.library.client.request.InsertRequest;
-import org.elasticsearch.action.index.IndexRequest;
-import org.elasticsearch.common.xcontent.XContentBuilder;
 
-public class ElasticSearchInsertRequest extends IndexRequest implements InsertRequest {
+@Getter
+public class IndexRequestWrapper implements InsertRequest {
+    private final IndexRequest request;
 
-    public ElasticSearchInsertRequest(String index, String type, String id) {
-        super(index, type, id);
-    }
-
-    @Override
-    public ElasticSearchInsertRequest source(XContentBuilder sourceBuilder) {
-        super.source(sourceBuilder);
-        return this;
+    public IndexRequestWrapper(String index, String type, String id,
+                               Map<String, Object> source) {
+        request = IndexRequest.builder()
+                              .index(index)
+                              .type(type)
+                              .id(id)
+                              .doc(source)
+                              .build();
     }
 }
diff --git a/oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/UpdateRequestWrapper.java b/oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/UpdateRequestWrapper.java
new file mode 100644
index 0000000..6459a8a
--- /dev/null
+++ b/oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/UpdateRequestWrapper.java
@@ -0,0 +1,38 @@
+/*
+ * 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.library.client.elasticsearch;
+
+import java.util.Map;
+import lombok.Getter;
+import org.apache.skywalking.oap.server.library.client.request.UpdateRequest;
+
+// TODO: unwrap
+@Getter
+public class UpdateRequestWrapper implements UpdateRequest {
+    private final org.apache.skywalking.library.elasticsearch.requests.UpdateRequest request;
+
+    public UpdateRequestWrapper(String index, String type, String id,
+                                Map<String, Object> source) {
+        request = org.apache.skywalking.library.elasticsearch.requests.UpdateRequest.builder()
+                                                                                    .index(index)
+                                                                                    .type(type)
+                                                                                    .id(id)
+                                                                                    .doc(source)
+                                                                                    .build();
+    }
+}
diff --git a/oap-server/server-library/library-client/src/test/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ITElasticSearchClient.java b/oap-server/server-library/library-client/src/test/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ITElasticSearchClient.java
index ae9a9a2..0f72515 100644
--- a/oap-server/server-library/library-client/src/test/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ITElasticSearchClient.java
+++ b/oap-server/server-library/library-client/src/test/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ITElasticSearchClient.java
@@ -18,26 +18,27 @@
 
 package org.apache.skywalking.oap.server.library.client.elasticsearch;
 
+import com.google.common.collect.ImmutableMap;
 import com.google.gson.Gson;
 import com.google.gson.JsonObject;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.TimeUnit;
 import org.apache.http.client.methods.HttpGet;
 import org.apache.skywalking.apm.util.StringUtil;
+import org.apache.skywalking.library.elasticsearch.bulk.BulkProcessor;
+import org.apache.skywalking.library.elasticsearch.response.Document;
 import org.elasticsearch.action.admin.indices.get.GetIndexRequest;
-import org.elasticsearch.action.bulk.BulkProcessor;
-import org.elasticsearch.action.get.GetResponse;
 import org.elasticsearch.action.index.IndexRequest;
 import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.client.Response;
 import org.elasticsearch.client.RestHighLevelClient;
-import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.common.xcontent.XContentFactory;
 import org.elasticsearch.index.query.QueryBuilders;
 import org.elasticsearch.search.builder.SearchSourceBuilder;
 import org.junit.After;
@@ -95,7 +96,7 @@ public class ITElasticSearchClient {
         properties.add("column1", column);
 
         String indexName = "test_index_operate";
-        client.createIndex(indexName, settings, doc);
+        // client.createIndex(indexName, settings, doc); // TODO
         Assert.assertTrue(client.isExistsIndex(indexName));
 
         JsonObject index = getIndex(indexName);
@@ -127,26 +128,25 @@ public class ITElasticSearchClient {
     public void documentOperate() throws IOException {
         String id = String.valueOf(System.currentTimeMillis());
 
-        XContentBuilder builder = XContentFactory.jsonBuilder()
-                                                 .startObject()
-                                                 .field("user", "kimchy")
-                                                 .field("post_date", "2009-11-15T14:12:12")
-                                                 .field("message", "trying out Elasticsearch")
-                                                 .endObject();
+        Map<String, Object> builder = ImmutableMap.<String, Object>builder()
+                                                 .put("user", "kimchy")
+                                                 .put("post_date", "2009-11-15T14:12:12")
+                                                 .put("message", "trying out Elasticsearch")
+                                                 .build();
 
         String indexName = "test_document_operate";
         client.forceInsert(indexName, id, builder);
 
-        GetResponse response = client.get(indexName, id);
-        Assert.assertEquals("kimchy", response.getSource().get("user"));
-        Assert.assertEquals("trying out Elasticsearch", response.getSource().get("message"));
+        Optional<Document> response = client.get(indexName, id);
+        Assert.assertEquals("kimchy", response.get().getSource().get("user"));
+        Assert.assertEquals("trying out Elasticsearch", response.get().getSource().get("message"));
 
-        builder = XContentFactory.jsonBuilder().startObject().field("user", "pengys").endObject();
+        builder = ImmutableMap.<String, Object>builder().put("user", "pengys").build();
         client.forceUpdate(indexName, id, builder);
 
         response = client.get(indexName, id);
-        Assert.assertEquals("pengys", response.getSource().get("user"));
-        Assert.assertEquals("trying out Elasticsearch", response.getSource().get("message"));
+        Assert.assertEquals("pengys", response.get().getSource().get("user"));
+        Assert.assertEquals("trying out Elasticsearch", response.get().getSource().get("message"));
 
         SearchSourceBuilder sourceBuilder = new SearchSourceBuilder();
         sourceBuilder.query(QueryBuilders.termQuery("user", "pengys"));
@@ -180,7 +180,7 @@ public class ITElasticSearchClient {
 
         Assert.assertTrue(client.isExistsTemplate(indexName));
 
-        XContentBuilder builder = XContentFactory.jsonBuilder().startObject().field("name", "pengys").endObject();
+        Map<String, Object> builder = ImmutableMap.of("name", "pengys");
         client.forceInsert(indexName + "-2019", "testid", builder);
         JsonObject index = getIndex(indexName + "-2019");
         LOGGER.info(index.toString());
@@ -208,13 +208,13 @@ public class ITElasticSearchClient {
         source.put("column2", "value2");
 
         for (int i = 0; i < 100; i++) {
-            IndexRequest indexRequest = new IndexRequest("bulk_insert_test", "type", String.valueOf(i));
-            indexRequest.source(source);
-            bulkProcessor.add(indexRequest);
+            // IndexRequest indexRequest = new IndexRequest("bulk_insert_test", "type", String.valueOf(i));
+            // indexRequest.source(source);
+            // bulkProcessor.add(indexRequest);
         }
 
-        bulkProcessor.flush();
-        bulkProcessor.awaitClose(2, TimeUnit.SECONDS);
+        // bulkProcessor.flush();
+        // bulkProcessor.awaitClose(2, TimeUnit.SECONDS);
     }
 
     @Test
@@ -235,12 +235,12 @@ public class ITElasticSearchClient {
 
         client.createOrUpdateTemplate(indexName, new HashMap<>(), mapping, 0);
 
-        XContentBuilder builder = XContentFactory.jsonBuilder().startObject().field("name", "pengys").endObject();
+        Map<String, Object> builder = ImmutableMap.of("name", "pengys");
         client.forceInsert(timeSeriesIndexName, "testid", builder);
 
-        List<String> indexes = client.retrievalIndexByAliases(indexName);
+        Collection<String> indexes = client.retrievalIndexByAliases(indexName);
         Assert.assertEquals(1, indexes.size());
-        String index = indexes.get(0);
+        String index = indexes.iterator().next();
         Assert.assertTrue(client.deleteByIndexName(index));
         Assert.assertFalse(client.isExistsIndex(timeSeriesIndexName));
         client.deleteTemplate(indexName);
diff --git a/oap-server/server-library/pom.xml b/oap-server/server-library/library-elasticsearch-client/pom.xml
similarity index 57%
copy from oap-server/server-library/pom.xml
copy to oap-server/server-library/library-elasticsearch-client/pom.xml
index acfcd44..7da3ed1 100644
--- a/oap-server/server-library/pom.xml
+++ b/oap-server/server-library/library-elasticsearch-client/pom.xml
@@ -17,27 +17,34 @@
   ~
   -->
 
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
     <parent>
-        <artifactId>oap-server</artifactId>
+        <artifactId>server-library</artifactId>
         <groupId>org.apache.skywalking</groupId>
         <version>8.8.0-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
-    <artifactId>server-library</artifactId>
-    <packaging>pom</packaging>
-    <modules>
-        <module>library-module</module>
-        <module>library-server</module>
-        <module>library-util</module>
-        <module>library-client</module>
-    </modules>
+    <artifactId>library-elasticsearch-client</artifactId>
 
     <dependencies>
         <dependency>
-            <groupId>org.apache.skywalking</groupId>
-            <artifactId>apm-util</artifactId>
+            <groupId>com.linecorp.armeria</groupId>
+            <artifactId>armeria</artifactId>
+            <version>1.11.0</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.testcontainers</groupId>
+            <artifactId>testcontainers</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.testcontainers</groupId>
+            <artifactId>elasticsearch</artifactId>
+            <scope>test</scope>
         </dependency>
     </dependencies>
-</project>
\ No newline at end of file
+</project>
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/ElasticSearchClient.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/ElasticSearchClient.java
new file mode 100644
index 0000000..edbaf50
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/ElasticSearchClient.java
@@ -0,0 +1,148 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.skywalking.library.elasticsearch;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.linecorp.armeria.client.ClientFactory;
+import com.linecorp.armeria.client.WebClient;
+import com.linecorp.armeria.client.WebClientBuilder;
+import com.linecorp.armeria.client.endpoint.EndpointGroup;
+import com.linecorp.armeria.common.HttpData;
+import com.linecorp.armeria.common.HttpStatus;
+import com.linecorp.armeria.common.SessionProtocol;
+import com.linecorp.armeria.common.auth.BasicToken;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.concurrent.CompletableFuture;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.apm.util.StringUtil;
+import org.apache.skywalking.library.elasticsearch.bulk.BulkProcessorBuilder;
+import org.apache.skywalking.library.elasticsearch.client.AliasClient;
+import org.apache.skywalking.library.elasticsearch.client.DocumentClient;
+import org.apache.skywalking.library.elasticsearch.client.IndexClient;
+import org.apache.skywalking.library.elasticsearch.client.SearchClient;
+import org.apache.skywalking.library.elasticsearch.client.TemplateClient;
+import org.apache.skywalking.library.elasticsearch.requests.factory.RequestFactory;
+import org.apache.skywalking.library.elasticsearch.requests.search.Search;
+import org.apache.skywalking.library.elasticsearch.response.NodeInfo;
+import org.apache.skywalking.library.elasticsearch.response.search.SearchResponse;
+
+@Slf4j
+public final class ElasticSearchClient implements AutoCloseable {
+    private final ObjectMapper mapper = new ObjectMapper()
+        .setSerializationInclusion(JsonInclude.Include.NON_NULL)
+        .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+
+    private final WebClient client;
+
+    private final ClientFactory clientFactory;
+
+    private final CompletableFuture<RequestFactory> requestFactory;
+
+    private final TemplateClient templateClient;
+
+    private final IndexClient indexClient;
+
+    private final DocumentClient documentClient;
+
+    private final AliasClient aliasClient;
+
+    private final SearchClient searchClient;
+
+    ElasticSearchClient(SessionProtocol protocol,
+                        String username, String password,
+                        EndpointGroup endpointGroup,
+                        ClientFactory clientFactory) {
+        this.clientFactory = clientFactory;
+        this.requestFactory = new CompletableFuture<>();
+
+        final WebClientBuilder builder = WebClient.builder(protocol, endpointGroup);
+        if (StringUtil.isNotBlank(username) && StringUtil.isNotBlank(password)) {
+            builder.auth(BasicToken.of(username, password));
+        }
+        builder.factory(clientFactory);
+        client = builder.build();
+
+        templateClient = new TemplateClient(requestFactory, client);
+        documentClient = new DocumentClient(requestFactory, client);
+        indexClient = new IndexClient(requestFactory, client);
+        aliasClient = new AliasClient(requestFactory, client);
+        searchClient = new SearchClient(requestFactory, client);
+    }
+
+    public static ElasticSearchClientBuilder builder() {
+        return new ElasticSearchClientBuilder();
+    }
+
+    public void connect() {
+        final CompletableFuture<Void> future =
+            client.get("/").aggregate().thenAcceptAsync(response -> {
+                final HttpStatus status = response.status();
+                if (status != HttpStatus.OK) {
+                    throw new RuntimeException(
+                        "Failed to connect to ElasticSearch server: " + status);
+                }
+                try (final HttpData content = response.content();
+                     final InputStream is = content.toInputStream()) {
+                    final NodeInfo node = mapper.readValue(is, NodeInfo.class);
+                    final String v = node.getVersion().getNumber();
+                    final ElasticSearchVersion version = ElasticSearchVersion.from(v);
+                    requestFactory.complete(RequestFactory.of(version));
+                } catch (IOException e) {
+                    throw new RuntimeException(e);
+                }
+            });
+        future.exceptionally(throwable -> {
+            log.error("Failed to determine ElasticSearch version", throwable);
+            requestFactory.completeExceptionally(throwable);
+            return null;
+        });
+        future.join();
+    }
+
+    public TemplateClient templates() {
+        return templateClient;
+    }
+
+    public DocumentClient documents() {
+        return documentClient;
+    }
+
+    public IndexClient index() {
+        return indexClient;
+    }
+
+    public AliasClient alias() {
+        return aliasClient;
+    }
+
+    public SearchResponse search(Search search, String... index) {
+        return searchClient.search(search, index);
+    }
+
+    public BulkProcessorBuilder bulkProcessor() {
+        return new BulkProcessorBuilder(requestFactory, client);
+    }
+
+    @Override
+    public void close() {
+        clientFactory.close();
+    }
+}
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/ElasticSearchClientBuilder.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/ElasticSearchClientBuilder.java
new file mode 100644
index 0000000..a9500c2
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/ElasticSearchClientBuilder.java
@@ -0,0 +1,150 @@
+/*
+ * 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.library.elasticsearch;
+
+import com.google.common.collect.ImmutableList;
+import com.linecorp.armeria.client.ClientFactory;
+import com.linecorp.armeria.client.ClientFactoryBuilder;
+import com.linecorp.armeria.client.Endpoint;
+import com.linecorp.armeria.client.endpoint.EndpointGroup;
+import com.linecorp.armeria.client.endpoint.healthcheck.HealthCheckedEndpointGroup;
+import com.linecorp.armeria.common.SessionProtocol;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.security.KeyStore;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import javax.net.ssl.TrustManagerFactory;
+import lombok.SneakyThrows;
+import org.apache.skywalking.apm.util.StringUtil;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
+import static org.apache.skywalking.apm.util.StringUtil.isNotBlank;
+
+public final class ElasticSearchClientBuilder {
+    private SessionProtocol protocol = SessionProtocol.HTTP;
+
+    private String username;
+
+    private String password;
+
+    private Duration healthCheckRetryInterval = Duration.ofSeconds(30);
+
+    private final ImmutableList.Builder<String> endpoints = ImmutableList.builder();
+
+    private String trustStorePath;
+
+    private String trustStorePass;
+
+    private Duration connectTimeout = Duration.ofMillis(500);
+
+    public ElasticSearchClientBuilder protocol(String protocol) {
+        checkArgument(isNotBlank(protocol), "protocol cannot be blank");
+        this.protocol = SessionProtocol.of(protocol);
+        return this;
+    }
+
+    public ElasticSearchClientBuilder username(String username) {
+        this.username = requireNonNull(username, "username");
+        return this;
+    }
+
+    public ElasticSearchClientBuilder password(String password) {
+        this.password = requireNonNull(password, "password");
+        return this;
+    }
+
+    public ElasticSearchClientBuilder endpoints(Iterable<String> endpoints) {
+        requireNonNull(endpoints, "endpoints");
+        this.endpoints.addAll(endpoints);
+        return this;
+    }
+
+    public ElasticSearchClientBuilder endpoints(String... endpoints) {
+        return endpoints(Arrays.asList(endpoints));
+    }
+
+    public ElasticSearchClientBuilder healthCheckRetryInterval(Duration healthCheckRetryInterval) {
+        requireNonNull(healthCheckRetryInterval, "healthCheckRetryInterval");
+        this.healthCheckRetryInterval = healthCheckRetryInterval;
+        return this;
+    }
+
+    public ElasticSearchClientBuilder trustStorePath(String trustStorePath) {
+        requireNonNull(trustStorePath, "trustStorePath");
+        this.trustStorePath = trustStorePath;
+        return this;
+    }
+
+    public ElasticSearchClientBuilder trustStorePass(String trustStorePass) {
+        requireNonNull(trustStorePass, "trustStorePass");
+        this.trustStorePass = trustStorePass;
+        return this;
+    }
+
+    public ElasticSearchClientBuilder connectTimeout(int connectTimeout) {
+        checkArgument(connectTimeout > 0, "connectTimeout must be positive");
+        this.connectTimeout = Duration.ofMillis(connectTimeout);
+        return this;
+    }
+
+    @SneakyThrows
+    public ElasticSearchClient build() {
+        final List<Endpoint> endpoints =
+            this.endpoints.build().stream().filter(StringUtil::isNotBlank).map(it -> {
+                final String[] parts = it.split(":", 2);
+                if (parts.length == 2) {
+                    return Endpoint.of(parts[0], Integer.parseInt(parts[1]));
+                }
+                return Endpoint.of(parts[0]);
+            }).collect(Collectors.toList());
+        final HealthCheckedEndpointGroup endpointGroup =
+            HealthCheckedEndpointGroup.builder(EndpointGroup.of(endpoints), "_cluster/health")
+                                      .protocol(protocol)
+                                      .useGet(true)
+                                      .retryInterval(healthCheckRetryInterval)
+                                      .build();
+        final ClientFactoryBuilder factoryBuilder = ClientFactory.builder();
+        factoryBuilder.connectTimeout(connectTimeout);
+
+        if (StringUtil.isNotBlank(trustStorePath)) {
+            final TrustManagerFactory trustManagerFactory =
+                TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm());
+            final KeyStore truststore = KeyStore.getInstance("jks");
+            try (final InputStream is = Files.newInputStream(Paths.get(trustStorePath))) {
+                truststore.load(is, trustStorePass.toCharArray());
+            }
+            trustManagerFactory.init(truststore);
+
+            factoryBuilder.tlsCustomizer(
+                sslContextBuilder -> sslContextBuilder.trustManager(trustManagerFactory));
+        }
+
+        return new ElasticSearchClient(
+            protocol,
+            username,
+            password,
+            endpointGroup,
+            factoryBuilder.build()
+        );
+    }
+}
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/ElasticSearchVersion.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/ElasticSearchVersion.java
new file mode 100644
index 0000000..777dc5c
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/ElasticSearchVersion.java
@@ -0,0 +1,80 @@
+/*
+ * 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.library.elasticsearch;
+
+import java.util.Objects;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import lombok.RequiredArgsConstructor;
+
+@RequiredArgsConstructor
+public final class ElasticSearchVersion implements Comparable<ElasticSearchVersion> {
+    public static final ElasticSearchVersion UNKNOWN = new ElasticSearchVersion(-1, -1);
+
+    public static final ElasticSearchVersion V6_0 = new ElasticSearchVersion(6, 0);
+
+    public static final ElasticSearchVersion V7_0 = new ElasticSearchVersion(7, 0);
+
+    public static final ElasticSearchVersion V8_0 = new ElasticSearchVersion(8, 0);
+
+    private final int major;
+
+    private final int minor;
+
+    @Override
+    public int compareTo(final ElasticSearchVersion o) {
+        if (major != o.major) {
+            return Integer.compare(major, o.major);
+        }
+        return Integer.compare(minor, o.minor);
+    }
+
+    @Override
+    public boolean equals(final Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        final ElasticSearchVersion that = (ElasticSearchVersion) o;
+        return major == that.major && minor == that.minor;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(major, minor);
+    }
+
+    @Override
+    public String toString() {
+        return major + "." + minor;
+    }
+
+    private static final Pattern REGEX = Pattern.compile("(\\d+)\\.(\\d+).*");
+
+    public static ElasticSearchVersion from(String version) {
+        final Matcher matcher = REGEX.matcher(version);
+        if (!matcher.matches()) {
+            throw new IllegalArgumentException("Failed to parse version: " + version);
+        }
+        final int major = Integer.parseInt(matcher.group(1));
+        final int minor = Integer.parseInt(matcher.group(2));
+        return new ElasticSearchVersion(major, minor);
+    }
+}
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/bulk/BulkProcessor.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/bulk/BulkProcessor.java
new file mode 100644
index 0000000..c0ad3c0
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/bulk/BulkProcessor.java
@@ -0,0 +1,130 @@
+/*
+ * 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.library.elasticsearch.bulk;
+
+import com.linecorp.armeria.client.WebClient;
+import com.linecorp.armeria.common.HttpStatus;
+import com.linecorp.armeria.common.util.Exceptions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.library.elasticsearch.requests.IndexRequest;
+import org.apache.skywalking.library.elasticsearch.requests.UpdateRequest;
+import org.apache.skywalking.library.elasticsearch.requests.factory.RequestFactory;
+
+import static java.util.Objects.requireNonNull;
+
+@Slf4j
+public final class BulkProcessor {
+    private final ArrayBlockingQueue<Object> requests;
+
+    private final CompletableFuture<RequestFactory> requestFactory;
+    private final WebClient client;
+
+    private final int bulkActions;
+    private final int concurrentRequests;
+
+    public BulkProcessor(
+        final CompletableFuture<RequestFactory> requestFactory,
+        final WebClient client, final int bulkActions,
+        final Duration flushInterval, final int concurrentRequests) {
+        this.requestFactory = requireNonNull(requestFactory, "requestFactory");
+        this.client = requireNonNull(client, "client");
+        this.bulkActions = bulkActions;
+
+        requireNonNull(flushInterval, "flushInterval");
+
+        this.concurrentRequests = concurrentRequests;
+        this.requests = new ArrayBlockingQueue<>(bulkActions + 1);
+        Executors.newSingleThreadScheduledExecutor(r -> {
+            final Thread thread = new Thread(r);
+            thread.setName("ElasticSearch BulkProcessor");
+            return thread;
+        }).scheduleWithFixedDelay(
+            this::flush,
+            0, flushInterval.getSeconds(),
+            TimeUnit.SECONDS
+        );
+    }
+
+    public BulkProcessor add(IndexRequest request) {
+        internalAdd(request);
+        return this;
+    }
+
+    public BulkProcessor add(UpdateRequest request) {
+        internalAdd(request);
+        return this;
+    }
+
+    private void internalAdd(Object request) {
+        requireNonNull(request, "request");
+        requests.add(request);
+        flushIfNeeded();
+    }
+
+    @SneakyThrows
+    private void flushIfNeeded() {
+        if (requests.size() >= bulkActions) {
+            flush();
+        }
+    }
+
+    private void flush() {
+        if (this.requests.isEmpty()) {
+            return;
+        }
+
+        final List<Object> requests = new ArrayList<>(this.requests.size());
+        this.requests.drainTo(requests);
+
+        log.debug("Executing bulk with {} requests", requests.size());
+
+        requestFactory.thenCompose(rf -> {
+            try {
+                final List<ByteBuf> bs = new ArrayList<>();
+                for (final Object request : requests) {
+                    bs.add(rf.codec().encode(request));
+                    bs.add(Unpooled.wrappedBuffer("\n".getBytes()));
+                }
+                final ByteBuf content = Unpooled.wrappedBuffer(bs.toArray(new ByteBuf[0]));
+                return client.execute(rf.bulk().bulk(content))
+                             .aggregate().thenAccept(response -> {
+                        final HttpStatus status = response.status();
+                        if (status != HttpStatus.OK) {
+                            throw new RuntimeException(
+                                "Failed to process bulk request: " + status);
+                        }
+                    });
+            } catch (Exception e) {
+                return Exceptions.throwUnsafely(e);
+            }
+        }).exceptionally(e -> {
+            log.error("Failed to execute bulk", e);
+            return null;
+        }).join();
+    }
+}
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/bulk/BulkProcessorBuilder.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/bulk/BulkProcessorBuilder.java
new file mode 100644
index 0000000..9c3d633
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/bulk/BulkProcessorBuilder.java
@@ -0,0 +1,63 @@
+/*
+ * 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.library.elasticsearch.bulk;
+
+import com.linecorp.armeria.client.WebClient;
+import java.time.Duration;
+import java.util.concurrent.CompletableFuture;
+import lombok.RequiredArgsConstructor;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.library.elasticsearch.requests.factory.RequestFactory;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
+
+@Slf4j
+@RequiredArgsConstructor
+public final class BulkProcessorBuilder {
+    private final CompletableFuture<RequestFactory> requestFactory;
+    private final WebClient client;
+
+    private int bulkActions = -1;
+    private Duration flushInterval;
+    private int concurrentRequests = 1;
+
+    // TODO: backoff
+
+    public BulkProcessorBuilder bulkActions(int bulkActions) {
+        checkArgument(bulkActions > 0, "bulkActions must be positive");
+        this.bulkActions = bulkActions;
+        return this;
+    }
+
+    public BulkProcessorBuilder flushInterval(Duration flushInterval) {
+        this.flushInterval = requireNonNull(flushInterval, "flushInterval");
+        return this;
+    }
+
+    public BulkProcessorBuilder concurrentRequests(int concurrentRequests) {
+        checkArgument(concurrentRequests > 0, "concurrentRequests must be positive");
+        this.concurrentRequests = concurrentRequests;
+        return this;
+    }
+
+    public BulkProcessor build() {
+        return new BulkProcessor(
+            requestFactory, client, bulkActions, flushInterval, concurrentRequests);
+    }
+}
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/client/AliasClient.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/client/AliasClient.java
new file mode 100644
index 0000000..91714f6
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/client/AliasClient.java
@@ -0,0 +1,64 @@
+/*
+ * 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.library.elasticsearch.client;
+
+import com.linecorp.armeria.client.WebClient;
+import com.linecorp.armeria.common.HttpData;
+import com.linecorp.armeria.common.HttpStatus;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import lombok.RequiredArgsConstructor;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.library.elasticsearch.requests.factory.RequestFactory;
+
+@Slf4j
+@RequiredArgsConstructor
+public final class AliasClient {
+    private final CompletableFuture<RequestFactory> requestFactory;
+
+    private final WebClient client;
+
+    @SneakyThrows
+    public Map<String, Object> indices(String name) {
+        // noinspection unchecked
+        return requestFactory.thenCompose(
+            rf -> client.execute(rf.alias().indices(name))
+                        .aggregate().thenApply(response -> {
+                    final HttpStatus status = response.status();
+                    if (status != HttpStatus.OK) {
+                        throw new RuntimeException("Failed to get alias indices: " + status);
+                    }
+
+                    try (final HttpData content = response.content();
+                         final InputStream is = content.toInputStream()) {
+                        return rf.codec().decode(is, Map.class); // TODO
+                    } catch (Exception e) {
+                        log.error("Failed to close input stream", e);
+                        return Collections.<String, Object>emptyMap();
+                    }
+
+                }).exceptionally(e -> {
+                    log.error("Failed to check whether index exists", e);
+                    return Collections.emptyMap();
+                })).get();
+    }
+}
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/client/DocumentClient.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/client/DocumentClient.java
new file mode 100644
index 0000000..b702004
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/client/DocumentClient.java
@@ -0,0 +1,129 @@
+/*
+ * 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.library.elasticsearch.client;
+
+import com.linecorp.armeria.client.WebClient;
+import com.linecorp.armeria.common.HttpData;
+import com.linecorp.armeria.common.HttpStatus;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import lombok.RequiredArgsConstructor;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.library.elasticsearch.requests.IndexRequest;
+import org.apache.skywalking.library.elasticsearch.requests.UpdateRequest;
+import org.apache.skywalking.library.elasticsearch.requests.factory.RequestFactory;
+import org.apache.skywalking.library.elasticsearch.response.Document;
+import org.apache.skywalking.library.elasticsearch.response.Documents;
+
+@Slf4j
+@RequiredArgsConstructor
+public final class DocumentClient {
+    private final CompletableFuture<RequestFactory> requestFactory;
+
+    private final WebClient client;
+
+    @SneakyThrows
+    public boolean exists(String index, String type, String id) {
+        return requestFactory.thenCompose(
+            rf -> client.execute(rf.document().exist(index, type, id))
+                        .aggregate().thenApply(response -> response.status() == HttpStatus.OK)
+                        .exceptionally(e -> {
+                            log.error("Failed to check whether document exists", e);
+                            return false;
+                        })).get();
+    }
+
+    @SneakyThrows
+    public Optional<Document> get(String index, String type, String id) {
+        return requestFactory.thenCompose(
+            rf -> client.execute(rf.document().get(index, type, id))
+                        .aggregate().thenApply(response -> {
+                    if (response.status() != HttpStatus.OK) {
+                        return Optional.<Document>empty();
+                    }
+
+                    try (final HttpData content = response.content();
+                         final InputStream is = content.toInputStream()) {
+                        return Optional.of(rf.codec().decode(is, Document.class));
+                    } catch (Exception e) {
+                        log.error("Failed to close input stream", e);
+                        return Optional.<Document>empty();
+                    }
+                })).get();
+    }
+
+    @SneakyThrows
+    public Optional<Documents> mget(String index, String type, Iterable<String> ids) {
+        return requestFactory.thenCompose(
+            rf -> client.execute(rf.document().mget(index, type, ids))
+                        .aggregate().thenApply(response -> {
+                    if (response.status() != HttpStatus.OK) {
+                        return Optional.<Documents>empty();
+                    }
+
+                    try (final HttpData content = response.content();
+                         final InputStream is = content.toInputStream()) {
+                        return Optional.of(rf.codec().decode(is, Documents.class));
+                    } catch (Exception e) {
+                        log.error("Failed to close input stream", e);
+                        return Optional.<Documents>empty();
+                    }
+                })).get();
+    }
+
+    @SneakyThrows
+    public void index(IndexRequest request, Map<String, Object> params) {
+        requestFactory.thenCompose(
+            rf -> client.execute(rf.document().index(request, params))
+                        .aggregate().thenAccept(response -> {
+                    final HttpStatus status = response.status();
+                    if (status != HttpStatus.CREATED && status != HttpStatus.OK) {
+                        throw new RuntimeException("Failed to index doc: " + status);
+                    }
+                })).join();
+    }
+
+    @SneakyThrows
+    public void update(UpdateRequest request, Map<String, Object> params) {
+        requestFactory.thenCompose(
+            rf -> client.execute(rf.document().update(request, params))
+                        .aggregate().thenAccept(response -> {
+                    final HttpStatus status = response.status();
+                    if (status != HttpStatus.OK) {
+                        throw new RuntimeException("Failed to update doc: " + status);
+                    }
+                })).join();
+    }
+
+    @SneakyThrows
+    public void delete(String index, String type, String query,
+                       Map<String, Object> params) {
+        requestFactory.thenCompose(
+            rf -> client.execute(rf.document().delete(index, type, query, params))
+                        .aggregate().thenAccept(response -> {
+                    final HttpStatus status = response.status();
+                    if (status != HttpStatus.OK) {
+                        throw new RuntimeException("Failed to delete docs by query: " + status);
+                    }
+                })).join();
+    }
+}
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/client/IndexClient.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/client/IndexClient.java
new file mode 100644
index 0000000..e7916a9
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/client/IndexClient.java
@@ -0,0 +1,106 @@
+/*
+ * 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.library.elasticsearch.client;
+
+import com.linecorp.armeria.client.WebClient;
+import com.linecorp.armeria.common.HttpData;
+import com.linecorp.armeria.common.HttpStatus;
+import java.io.InputStream;
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import lombok.RequiredArgsConstructor;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.library.elasticsearch.requests.factory.RequestFactory;
+
+@Slf4j
+@RequiredArgsConstructor
+public final class IndexClient {
+    private final CompletableFuture<RequestFactory> requestFactory;
+
+    private final WebClient client;
+
+    @SneakyThrows
+    public boolean exists(String name) {
+        return requestFactory.thenCompose(
+            rf -> client.execute(rf.index().exists(name))
+                        .aggregate().thenApply(response -> response.status() == HttpStatus.OK)
+                        .exceptionally(e -> {
+                            log.error("Failed to check whether index exists", e);
+                            return false;
+                        })).get();
+    }
+
+    @SneakyThrows
+    public Map<String, Object> get(String name) {
+        final CompletableFuture<Map<String, Object>> future = requestFactory.thenCompose(
+            rf -> client.execute(rf.index().get(name))
+                        .aggregate().thenApply(response -> {
+                    final HttpStatus status = response.status();
+                    if (status == HttpStatus.NOT_FOUND) {
+                        return Collections.emptyMap();
+                    }
+
+                    try (final HttpData content = response.content();
+                         final InputStream is = content.toInputStream()) {
+                        // noinspection unchecked
+                        return (Map<String, Object>) rf.codec().decode(is, Map.class); // TODO
+                    } catch (Exception e) {
+                        log.error("Failed to close input stream", e);
+                        return Collections.emptyMap();
+                    }
+                }));
+
+        return future.get();
+    }
+
+    @SneakyThrows
+    public boolean create(String name) {
+        return requestFactory.thenCompose(
+            rf -> client.execute(rf.index().create(name))
+                        .aggregate().thenApply(response -> response.status() == HttpStatus.OK)
+                        .exceptionally(e -> {
+                            log.error("Failed to check whether index exists", e);
+                            return false;
+                        })).get();
+    }
+
+    @SneakyThrows
+    public boolean delete(String name) {
+        return requestFactory.thenCompose(
+            rf -> client.execute(rf.index().delete(name))
+                        .aggregate().thenApply(response -> response.status() == HttpStatus.OK)
+                        .exceptionally(e -> {
+                            log.error("Failed to delete whether index exists", e);
+                            return false;
+                        })).get();
+    }
+
+    @SneakyThrows
+    public boolean putMapping(String name, String type,
+                              Map<String, Object> mapping) {
+        return requestFactory.thenCompose(
+            rf -> client.execute(rf.index().putMapping(name, type, mapping))
+                        .aggregate().thenApply(response -> response.status() == HttpStatus.OK)
+                        .exceptionally(e -> {
+                            log.error("Failed to update index mapping", e);
+                            return false;
+                        })).get();
+    }
+}
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/client/SearchClient.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/client/SearchClient.java
new file mode 100644
index 0000000..b8d9d08
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/client/SearchClient.java
@@ -0,0 +1,63 @@
+/*
+ * 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.library.elasticsearch.client;
+
+import com.linecorp.armeria.client.WebClient;
+import com.linecorp.armeria.common.HttpData;
+import com.linecorp.armeria.common.HttpStatus;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.concurrent.CompletableFuture;
+import lombok.RequiredArgsConstructor;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.library.elasticsearch.requests.factory.RequestFactory;
+import org.apache.skywalking.library.elasticsearch.requests.search.Search;
+import org.apache.skywalking.library.elasticsearch.response.search.SearchResponse;
+
+@Slf4j
+@RequiredArgsConstructor
+public final class SearchClient {
+    private final CompletableFuture<RequestFactory> requestFactory;
+
+    private final WebClient client;
+
+    @SneakyThrows
+    public SearchResponse search(Search criteria, String... index) {
+        return requestFactory.thenCompose(
+            rf -> client.execute(rf.search().search(criteria, index))
+                        .aggregate().thenApply(response -> {
+                    if (response.status() != HttpStatus.OK) {
+                        throw new RuntimeException(
+                            "Failed to search documents: " + Arrays.toString(index) + ". " +
+                                response.contentUtf8());
+                    }
+
+                    try (final HttpData content = response.content();
+                         final InputStream is = content.toInputStream()) {
+                        return rf.codec().decode(is, SearchResponse.class);
+                    } catch (Exception e) {
+                        log.error("Failed to close input stream", e);
+                        return new SearchResponse();
+                    }
+                }).exceptionally(e -> {
+                    log.error("Failed to search documents", e);
+                    return new SearchResponse();
+                })).get();
+    }
+}
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/client/TemplateClient.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/client/TemplateClient.java
new file mode 100644
index 0000000..9129270
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/client/TemplateClient.java
@@ -0,0 +1,106 @@
+/*
+ * 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.library.elasticsearch.client;
+
+import com.linecorp.armeria.client.WebClient;
+import com.linecorp.armeria.common.HttpData;
+import com.linecorp.armeria.common.HttpStatus;
+import java.io.InputStream;
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import lombok.RequiredArgsConstructor;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.library.elasticsearch.requests.factory.RequestFactory;
+
+@Slf4j
+@RequiredArgsConstructor
+public final class TemplateClient {
+    private final CompletableFuture<RequestFactory> requestFactory;
+
+    private final WebClient client;
+
+    @SneakyThrows
+    public boolean exists(String name) {
+        return requestFactory.thenCompose(
+            rf -> client.execute(rf.template().exists(name))
+                        .aggregate().thenApply(response -> {
+                    final HttpStatus status = response.status();
+                    if (status == HttpStatus.OK) {
+                        return true;
+                    } else if (status == HttpStatus.NOT_FOUND) {
+                        return false;
+                    }
+                    throw new RuntimeException(
+                        "Response status code of template exists request should be 200 or 404," +
+                            " but it was: " + status.code());
+                }).exceptionally(e -> {
+                    log.error("Failed to check whether template exists", e);
+                    return false;
+                })).get();
+    }
+
+    @SneakyThrows
+    public Map<String, Object> get(String name) {
+        //noinspection unchecked
+        return requestFactory.thenCompose(
+            rf -> client.execute(rf.template().get(name))
+                        .aggregate().thenApply(response -> {
+                    final HttpStatus status = response.status();
+                    if (status != HttpStatus.OK) {
+                        throw new RuntimeException("Failed to get template: " + status);
+                    }
+
+                    try (final HttpData content = response.content();
+                         final InputStream is = content.toInputStream()) {
+                        return rf.codec().decode(is, Map.class); // TODO
+                    } catch (Exception e) {
+                        log.error("Failed to close input stream", e);
+                        return Collections.<String, Object>emptyMap();
+                    }
+                }).exceptionally(e -> {
+                    log.error("Failed to check whether template exists", e);
+                    return Collections.emptyMap();
+                })).get();
+    }
+
+    @SneakyThrows
+    public boolean delete(String name) {
+        return requestFactory.thenCompose(
+            rf -> client.execute(rf.template().delete(name))
+                        .aggregate().thenApply(response -> response.status() == HttpStatus.OK)
+                        .exceptionally(e -> {
+                            log.error("Failed to check whether template exists", e);
+                            return false;
+                        })).get();
+    }
+
+    @SneakyThrows
+    public boolean createOrUpdate(String name, Map<String, Object> settings,
+                                  Map<String, Object> mapping, int order) {
+        return requestFactory.thenCompose(
+            rf -> client.execute(rf.template().createOrUpdate(name, settings, mapping, order))
+                        .aggregate().thenApply(response -> response.status() == HttpStatus.OK)
+                        .exceptionally(e -> {
+                            log.error("Failed to create or update template", e);
+                            return false;
+                        })).get();
+    }
+
+}
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/IndexRequest.java
similarity index 61%
copy from oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
copy to oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/IndexRequest.java
index ebad19e..c51e6bf 100644
--- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/IndexRequest.java
@@ -13,27 +13,21 @@
  * 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.core.storage.type;
-
-/**
- * StorageDataComplexObject implementation supports String-Object interconversion.
- */
-public interface StorageDataComplexObject<T> {
-    /**
-     * @return string representing this object.
-     */
-    String toStorageData();
+package org.apache.skywalking.library.elasticsearch.requests;
 
-    /**
-     * Initialize this object based on the given string data.
-     */
-    void toObject(String data);
+import java.util.Map;
+import lombok.Builder;
+import lombok.Getter;
+import lombok.Setter;
 
-    /**
-     * Initialize the object based on the given source.
-     */
-    void copyFrom(T source);
+@Getter
+@Setter
+@Builder
+public final class IndexRequest {
+    private final String index;
+    private final String type;
+    private final String id;
+    private final Map<String, Object> doc;
 }
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/UpdateRequest.java
similarity index 61%
copy from oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
copy to oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/UpdateRequest.java
index ebad19e..37bed7e 100644
--- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/UpdateRequest.java
@@ -13,27 +13,21 @@
  * 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.core.storage.type;
-
-/**
- * StorageDataComplexObject implementation supports String-Object interconversion.
- */
-public interface StorageDataComplexObject<T> {
-    /**
-     * @return string representing this object.
-     */
-    String toStorageData();
+package org.apache.skywalking.library.elasticsearch.requests;
 
-    /**
-     * Initialize this object based on the given string data.
-     */
-    void toObject(String data);
+import java.util.Map;
+import lombok.Builder;
+import lombok.Getter;
+import lombok.Setter;
 
-    /**
-     * Initialize the object based on the given source.
-     */
-    void copyFrom(T source);
+@Getter
+@Setter
+@Builder
+public final class UpdateRequest {
+    private final String index;
+    private final String type;
+    private final String id;
+    private final Map<String, Object> doc;
 }
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/AliasFactory.java
similarity index 62%
copy from oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
copy to oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/AliasFactory.java
index ebad19e..6ccf6b8 100644
--- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/AliasFactory.java
@@ -13,27 +13,15 @@
  * 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.core.storage.type;
+package org.apache.skywalking.library.elasticsearch.requests.factory;
 
-/**
- * StorageDataComplexObject implementation supports String-Object interconversion.
- */
-public interface StorageDataComplexObject<T> {
-    /**
-     * @return string representing this object.
-     */
-    String toStorageData();
-
-    /**
-     * Initialize this object based on the given string data.
-     */
-    void toObject(String data);
+import com.linecorp.armeria.common.HttpRequest;
 
+public interface AliasFactory {
     /**
-     * Initialize the object based on the given source.
+     * Returns a request to list all indices behind the {@code alias}.
      */
-    void copyFrom(T source);
+    HttpRequest indices(String alias);
 }
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/BulkFactory.java
similarity index 62%
copy from oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
copy to oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/BulkFactory.java
index ebad19e..9b64ab1 100644
--- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/BulkFactory.java
@@ -13,27 +13,16 @@
  * 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.core.storage.type;
+package org.apache.skywalking.library.elasticsearch.requests.factory;
 
-/**
- * StorageDataComplexObject implementation supports String-Object interconversion.
- */
-public interface StorageDataComplexObject<T> {
-    /**
-     * @return string representing this object.
-     */
-    String toStorageData();
-
-    /**
-     * Initialize this object based on the given string data.
-     */
-    void toObject(String data);
+import com.linecorp.armeria.common.HttpRequest;
+import io.netty.buffer.ByteBuf;
 
+public interface BulkFactory {
     /**
-     * Initialize the object based on the given source.
+     * Returns a request to perform multiple indexing or delete operations in a single API call.
      */
-    void copyFrom(T source);
+    HttpRequest bulk(ByteBuf content);
 }
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/Codec.java
similarity index 62%
copy from oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
copy to oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/Codec.java
index ebad19e..da7c271 100644
--- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/Codec.java
@@ -13,27 +13,18 @@
  * 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.core.storage.type;
+package org.apache.skywalking.library.elasticsearch.requests.factory;
+
+import io.netty.buffer.ByteBuf;
+import java.io.InputStream;
 
 /**
- * StorageDataComplexObject implementation supports String-Object interconversion.
+ * Responsible to encode requests and decode responses.
  */
-public interface StorageDataComplexObject<T> {
-    /**
-     * @return string representing this object.
-     */
-    String toStorageData();
-
-    /**
-     * Initialize this object based on the given string data.
-     */
-    void toObject(String data);
+public interface Codec {
+    ByteBuf encode(Object request) throws Exception;
 
-    /**
-     * Initialize the object based on the given source.
-     */
-    void copyFrom(T source);
+    <T> T decode(InputStream inputStream, Class<T> type) throws Exception;
 }
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/DocumentFactory.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/DocumentFactory.java
new file mode 100644
index 0000000..b60f682
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/DocumentFactory.java
@@ -0,0 +1,56 @@
+/*
+ * 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.library.elasticsearch.requests.factory;
+
+import com.linecorp.armeria.common.HttpRequest;
+import java.util.Map;
+import org.apache.skywalking.library.elasticsearch.requests.IndexRequest;
+import org.apache.skywalking.library.elasticsearch.requests.UpdateRequest;
+
+public interface DocumentFactory {
+    /**
+     * Returns a request to check whether the document exists in the {@code index} or not.
+     */
+    HttpRequest exist(String index, String type, String id);
+
+    /**
+     * Returns a request to get a document of {@code id} in {@code index}.
+     */
+    HttpRequest get(String index, String type, String id);
+
+    /**
+     * Returns a request to get multiple documents of {@code ids} in {@code index}.
+     */
+    HttpRequest mget(String index, String type, Iterable<String> ids);
+
+    /**
+     * Returns a request to index a document with {@link IndexRequest}.
+     */
+    HttpRequest index(IndexRequest request, Map<String, Object> params);
+
+    /**
+     * Returns a request to update a document with {@link UpdateRequest}.
+     */
+    HttpRequest update(UpdateRequest request, Map<String, Object> params);
+
+    /**
+     * Returns a request to delete documents matching the given {@code query} in {@code index}.
+     */
+    HttpRequest delete(String index, String type, String query,
+                       Map<String, Object> params);
+}
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/IndexFactory.java
similarity index 50%
copy from oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
copy to oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/IndexFactory.java
index ebad19e..35e9567 100644
--- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/IndexFactory.java
@@ -13,27 +13,37 @@
  * 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.core.storage.type;
+package org.apache.skywalking.library.elasticsearch.requests.factory;
+
+import com.linecorp.armeria.common.HttpRequest;
+import java.util.Map;
+
+public interface IndexFactory {
+    /**
+     * Returns a request to check whether the {@code index} exists or not.
+     */
+    HttpRequest exists(String index);
+
+    /**
+     * Returns a request to get an index of name {@code index}.
+     */
+    HttpRequest get(String index);
 
-/**
- * StorageDataComplexObject implementation supports String-Object interconversion.
- */
-public interface StorageDataComplexObject<T> {
     /**
-     * @return string representing this object.
+     * Returns a request to create an index of name {@code index}.
      */
-    String toStorageData();
+    HttpRequest create(String index);
 
     /**
-     * Initialize this object based on the given string data.
+     * Returns a request to delete an index of name {@code index}.
      */
-    void toObject(String data);
+    HttpRequest delete(String index);
 
     /**
-     * Initialize the object based on the given source.
+     * Returns a request to update the {@code mapping} of an index of name {@code index}.
      */
-    void copyFrom(T source);
+    HttpRequest putMapping(String index, String type,
+                           Map<String, Object> mapping);
 }
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/RequestFactory.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/RequestFactory.java
new file mode 100644
index 0000000..8d685b1
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/RequestFactory.java
@@ -0,0 +1,91 @@
+/*
+ * 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.library.elasticsearch.requests.factory;
+
+import org.apache.skywalking.library.elasticsearch.ElasticSearchVersion;
+import org.apache.skywalking.library.elasticsearch.requests.factory.v6.V6RequestFactory;
+
+import static java.util.Objects.requireNonNull;
+import static org.apache.skywalking.library.elasticsearch.ElasticSearchVersion.V6_0;
+import static org.apache.skywalking.library.elasticsearch.ElasticSearchVersion.V7_0;
+import static org.apache.skywalking.library.elasticsearch.ElasticSearchVersion.V8_0;
+
+public interface RequestFactory {
+    /**
+     * Returns a {@link RequestFactory} that is responsible to compose correct requests according to
+     * the syntax of specific {@link ElasticSearchVersion}.
+     */
+    static RequestFactory of(ElasticSearchVersion version) {
+        requireNonNull(version, "version");
+        if (version.compareTo(V6_0) >= 0 && version.compareTo(V7_0) < 0) {
+            return V6RequestFactory.INSTANCE;
+        }
+        if (version.compareTo(V7_0) >= 0 && version.compareTo(V8_0) < 0) {
+            return V6RequestFactory.INSTANCE;
+        }
+
+        throw new UnsupportedOperationException("Version " + version + " is not supported.");
+    }
+
+    /**
+     * Returns a {@link TemplateFactory} that is dedicated to compose template-related requests.
+     *
+     * @see TemplateFactory
+     */
+    TemplateFactory template();
+
+    /**
+     * Returns a {@link IndexFactory} that is dedicated to compose index-related requests.
+     *
+     * @see IndexFactory
+     */
+    IndexFactory index();
+
+    /**
+     * Returns a {@link AliasFactory} that is dedicated to compose alias-related requests.
+     *
+     * @see AliasFactory
+     */
+    AliasFactory alias();
+
+    /**
+     * Returns a {@link DocumentFactory} that is dedicated to compose document-related requests.
+     *
+     * @see DocumentFactory
+     */
+    DocumentFactory document();
+
+    /**
+     * Returns a {@link SearchFactory} that is dedicated to compose searching-related requests.
+     *
+     * @see DocumentFactory
+     */
+    SearchFactory search();
+
+    /**
+     * Returns a {@link SearchFactory} that is dedicated to compose bulk-related requests.
+     *
+     * @see DocumentFactory
+     */
+    BulkFactory bulk();
+
+    /**
+     * Returns a {@link Codec} to encode the requests and decode the response.
+     */
+    Codec codec();
+}
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/SearchFactory.java
similarity index 62%
copy from oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
copy to oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/SearchFactory.java
index ebad19e..30a8e7a 100644
--- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/SearchFactory.java
@@ -13,27 +13,16 @@
  * 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.core.storage.type;
+package org.apache.skywalking.library.elasticsearch.requests.factory;
 
-/**
- * StorageDataComplexObject implementation supports String-Object interconversion.
- */
-public interface StorageDataComplexObject<T> {
-    /**
-     * @return string representing this object.
-     */
-    String toStorageData();
-
-    /**
-     * Initialize this object based on the given string data.
-     */
-    void toObject(String data);
+import com.linecorp.armeria.common.HttpRequest;
+import org.apache.skywalking.library.elasticsearch.requests.search.Search;
 
+public interface SearchFactory {
     /**
-     * Initialize the object based on the given source.
+     * Returns a request to search documents.
      */
-    void copyFrom(T source);
+    HttpRequest search(Search search, String... index);
 }
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/TemplateFactory.java
similarity index 52%
copy from oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
copy to oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/TemplateFactory.java
index ebad19e..9726f1f 100644
--- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/TemplateFactory.java
@@ -13,27 +13,33 @@
  * 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.core.storage.type;
+package org.apache.skywalking.library.elasticsearch.requests.factory;
+
+import com.linecorp.armeria.common.HttpRequest;
+import java.util.Map;
+
+public interface TemplateFactory {
+    /**
+     * Returns a request to check whether the template exists or not.
+     */
+    HttpRequest exists(String name);
 
-/**
- * StorageDataComplexObject implementation supports String-Object interconversion.
- */
-public interface StorageDataComplexObject<T> {
     /**
-     * @return string representing this object.
+     * Returns a request to get a template of {@code name}.
      */
-    String toStorageData();
+    HttpRequest get(String name);
 
     /**
-     * Initialize this object based on the given string data.
+     * Returns a request to delete a template of {@code name}.
      */
-    void toObject(String data);
+    HttpRequest delete(String name);
 
     /**
-     * Initialize the object based on the given source.
+     * Returns a request to create or update a template of {@code name} with the given {@code
+     * settings}, {@code mapping} and {@code order}.
      */
-    void copyFrom(T source);
+    HttpRequest createOrUpdate(String name, Map<String, Object> settings,
+                               Map<String, Object> mapping, int order);
 }
diff --git a/oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ElasticSearchInsertRequest.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/V6AliasFactory.java
similarity index 53%
copy from oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ElasticSearchInsertRequest.java
copy to oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/V6AliasFactory.java
index 9c0655c..f22a0e5 100644
--- a/oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ElasticSearchInsertRequest.java
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/V6AliasFactory.java
@@ -15,21 +15,24 @@
  * limitations under the License.
  */
 
-package org.apache.skywalking.oap.server.library.client.elasticsearch;
+package org.apache.skywalking.library.elasticsearch.requests.factory.v6;
 
-import org.apache.skywalking.oap.server.library.client.request.InsertRequest;
-import org.elasticsearch.action.index.IndexRequest;
-import org.elasticsearch.common.xcontent.XContentBuilder;
+import com.google.common.base.Strings;
+import com.linecorp.armeria.common.HttpRequest;
+import org.apache.skywalking.library.elasticsearch.requests.factory.AliasFactory;
 
-public class ElasticSearchInsertRequest extends IndexRequest implements InsertRequest {
+import static com.google.common.base.Preconditions.checkArgument;
 
-    public ElasticSearchInsertRequest(String index, String type, String id) {
-        super(index, type, id);
-    }
+final class V6AliasFactory implements AliasFactory {
+    static final AliasFactory INSTANCE = new V6AliasFactory();
 
     @Override
-    public ElasticSearchInsertRequest source(XContentBuilder sourceBuilder) {
-        super.source(sourceBuilder);
-        return this;
+    public HttpRequest indices(String alias) {
+        checkArgument(!Strings.isNullOrEmpty(alias), "alias cannot be null or empty");
+
+        return HttpRequest.builder()
+                          .get("/_alias/{name}")
+                          .pathParam("name", alias)
+                          .build();
     }
 }
diff --git a/oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ElasticSearchInsertRequest.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/V6BulkFactory.java
similarity index 53%
copy from oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ElasticSearchInsertRequest.java
copy to oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/V6BulkFactory.java
index 9c0655c..3bfcac1 100644
--- a/oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ElasticSearchInsertRequest.java
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/V6BulkFactory.java
@@ -15,21 +15,24 @@
  * limitations under the License.
  */
 
-package org.apache.skywalking.oap.server.library.client.elasticsearch;
+package org.apache.skywalking.library.elasticsearch.requests.factory.v6;
 
-import org.apache.skywalking.oap.server.library.client.request.InsertRequest;
-import org.elasticsearch.action.index.IndexRequest;
-import org.elasticsearch.common.xcontent.XContentBuilder;
+import com.linecorp.armeria.common.HttpData;
+import com.linecorp.armeria.common.HttpRequest;
+import com.linecorp.armeria.common.MediaType;
+import io.netty.buffer.ByteBuf;
+import lombok.SneakyThrows;
+import org.apache.skywalking.library.elasticsearch.requests.factory.BulkFactory;
 
-public class ElasticSearchInsertRequest extends IndexRequest implements InsertRequest {
-
-    public ElasticSearchInsertRequest(String index, String type, String id) {
-        super(index, type, id);
-    }
+final class V6BulkFactory implements BulkFactory {
+    static final BulkFactory INSTANCE = new V6BulkFactory();
 
+    @SneakyThrows
     @Override
-    public ElasticSearchInsertRequest source(XContentBuilder sourceBuilder) {
-        super.source(sourceBuilder);
-        return this;
+    public HttpRequest bulk(ByteBuf content) {
+        return HttpRequest.builder()
+                          .post("/_bulk")
+                          .content(MediaType.JSON, HttpData.wrap(content))
+                          .build();
     }
 }
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/V6DocumentFactory.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/V6DocumentFactory.java
new file mode 100644
index 0000000..d72022b
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/V6DocumentFactory.java
@@ -0,0 +1,160 @@
+/*
+ * 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.library.elasticsearch.requests.factory.v6;
+
+import com.google.common.collect.ImmutableMap;
+import com.linecorp.armeria.common.HttpRequest;
+import com.linecorp.armeria.common.HttpRequestBuilder;
+import com.linecorp.armeria.common.MediaType;
+import java.util.Map;
+import lombok.SneakyThrows;
+import org.apache.skywalking.library.elasticsearch.requests.IndexRequest;
+import org.apache.skywalking.library.elasticsearch.requests.UpdateRequest;
+import org.apache.skywalking.library.elasticsearch.requests.factory.DocumentFactory;
+import org.apache.skywalking.library.elasticsearch.requests.factory.v6.codec.V6Codec;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Strings.isNullOrEmpty;
+import static com.google.common.collect.Iterables.isEmpty;
+import static java.util.Objects.requireNonNull;
+
+final class V6DocumentFactory implements DocumentFactory {
+    static final DocumentFactory INSTANCE = new V6DocumentFactory();
+
+    @Override
+    public HttpRequest exist(String index, String type, String id) {
+        checkArgument(!isNullOrEmpty(index), "index cannot be null or empty");
+        checkArgument(!isNullOrEmpty(type), "type cannot be null or empty");
+        checkArgument(!isNullOrEmpty(id), "id cannot be null or empty");
+
+        return HttpRequest.builder()
+                          .head("/{index}/{type}/{id}")
+                          .pathParam("index", index)
+                          .pathParam("type", type)
+                          .pathParam("id", id)
+                          .build();
+    }
+
+    @Override
+    public HttpRequest get(String index, String type, String id) {
+        checkArgument(!isNullOrEmpty(index), "index cannot be null or empty");
+        checkArgument(!isNullOrEmpty(type), "type cannot be null or empty");
+        checkArgument(!isNullOrEmpty(id), "id cannot be null or empty");
+
+        return HttpRequest.builder()
+                          .get("/{index}/{type}/{id}")
+                          .pathParam("index", index)
+                          .pathParam("type", type)
+                          .pathParam("id", id)
+                          .build();
+    }
+
+    @SneakyThrows
+    @Override
+    public HttpRequest mget(String index, String type, Iterable<String> ids) {
+        checkArgument(!isNullOrEmpty(index), "index cannot be null or empty");
+        checkArgument(!isNullOrEmpty(type), "type cannot be null or empty");
+        checkArgument(ids != null && !isEmpty(ids), "ids cannot be null or empty");
+
+        final Map<String, Iterable<String>> m = ImmutableMap.of("ids", ids);
+        final byte[] content = V6Codec.MAPPER.writeValueAsBytes(m);
+        return HttpRequest.builder()
+                          .get("/{index}/{type}/_mget")
+                          .pathParam("index", index)
+                          .pathParam("type", type)
+                          .content(MediaType.JSON, content)
+                          .build();
+    }
+
+    @SneakyThrows
+    @Override
+    public HttpRequest index(IndexRequest request, Map<String, Object> params) {
+        requireNonNull(request, "request");
+
+        final String index = request.getIndex();
+        final String type = request.getType();
+        final String id = request.getId();
+        final Map<String, Object> doc = request.getDoc();
+
+        checkArgument(!isNullOrEmpty(index), "request.index cannot be null or empty");
+        checkArgument(!isNullOrEmpty(type), "request.type cannot be null or empty");
+        checkArgument(!isNullOrEmpty(id), "request.id cannot be null or empty");
+
+        final HttpRequestBuilder builder = HttpRequest.builder();
+        if (params != null) {
+            params.forEach(builder::queryParam);
+        }
+        final byte[] content = V6Codec.MAPPER.writeValueAsBytes(doc);
+
+        builder.put("/{index}/{type}/{id}")
+               .pathParam("index", index)
+               .pathParam("type", type)
+               .pathParam("id", id)
+               .content(MediaType.JSON, content);
+
+        return builder.build();
+    }
+
+    @SneakyThrows
+    @Override
+    public HttpRequest update(UpdateRequest request, Map<String, Object> params) {
+        requireNonNull(request, "request");
+
+        final String index = request.getIndex();
+        final String type = request.getType();
+        final String id = request.getId();
+        final Map<String, Object> doc = request.getDoc();
+
+        checkArgument(!isNullOrEmpty(index), "index cannot be null or empty");
+        checkArgument(!isNullOrEmpty(type), "type cannot be null or empty");
+        checkArgument(!isNullOrEmpty(id), "id cannot be null or empty");
+        checkArgument(doc != null && !isEmpty(doc.entrySet()), "doc cannot be null or empty");
+
+        final HttpRequestBuilder builder = HttpRequest.builder();
+        if (params != null) {
+            params.forEach(builder::queryParam);
+        }
+        final byte[] content = V6Codec.MAPPER.writeValueAsBytes(doc);
+
+        builder.put("/{index}/{type}/{id}")
+               .pathParam("index", index)
+               .pathParam("type", type)
+               .pathParam("id", id)
+               .content(MediaType.JSON, content);
+
+        return builder.build();
+    }
+
+    @Override
+    public HttpRequest delete(String index, String type, String query,
+                              Map<String, Object> params) {
+        checkArgument(!isNullOrEmpty(index), "index cannot be null or empty");
+        checkArgument(!isNullOrEmpty(type), "type cannot be null or empty");
+        checkArgument(!isNullOrEmpty(query), "query cannot be null or empty");
+
+        final HttpRequestBuilder builder = HttpRequest.builder();
+        if (params != null) {
+            params.forEach(builder::queryParam);
+        }
+
+        return builder.delete("/{index}/{type}/_delete_by_query")
+                      .pathParam("index", index)
+                      .pathParam("type", type)
+                      .build();
+    }
+}
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/V6IndexFactory.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/V6IndexFactory.java
new file mode 100644
index 0000000..9805333
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/V6IndexFactory.java
@@ -0,0 +1,91 @@
+/*
+ * 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.library.elasticsearch.requests.factory.v6;
+
+import com.google.common.base.Strings;
+import com.linecorp.armeria.common.HttpRequest;
+import com.linecorp.armeria.common.MediaType;
+import java.util.Map;
+import lombok.SneakyThrows;
+import org.apache.skywalking.library.elasticsearch.requests.factory.IndexFactory;
+import org.apache.skywalking.library.elasticsearch.requests.factory.v6.codec.V6Codec;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+final class V6IndexFactory implements IndexFactory {
+    static final IndexFactory INSTANCE = new V6IndexFactory();
+
+    @Override
+    public HttpRequest exists(String index) {
+        checkArgument(!Strings.isNullOrEmpty(index), "index cannot be null or empty");
+
+        return HttpRequest.builder()
+                          .head("/{index}")
+                          .pathParam("index", index)
+                          .build();
+    }
+
+    @Override
+    public HttpRequest get(final String index) {
+        checkArgument(!Strings.isNullOrEmpty(index), "index cannot be null or empty");
+
+        return HttpRequest.builder()
+                          .get("/{index}")
+                          .pathParam("index", index)
+                          .build();
+    }
+
+    @SneakyThrows
+    @Override
+    public HttpRequest create(String index) {
+        checkArgument(!Strings.isNullOrEmpty(index), "index cannot be null or empty");
+
+        return HttpRequest.builder()
+                          .put("/{index}")
+                          .pathParam("index", index)
+                          .build();
+    }
+
+    @Override
+    public HttpRequest delete(String index) {
+        checkArgument(!Strings.isNullOrEmpty(index), "index cannot be null or empty");
+
+        return HttpRequest.builder()
+                          .delete("/{index}")
+                          .pathParam("index", index)
+                          .build();
+    }
+
+    @SneakyThrows
+    @Override
+    @SuppressWarnings("unchecked")
+    public HttpRequest putMapping(String index, String type,
+                                  Map<String, Object> mapping) {
+        checkArgument(!Strings.isNullOrEmpty(index), "index cannot be null or empty");
+        checkArgument(!Strings.isNullOrEmpty(type), "type cannot be null or empty");
+
+        final Map<String, Object> properties = (Map<String, Object>) mapping.get(type);
+        final byte[] content = V6Codec.MAPPER.writeValueAsBytes(properties);
+        return HttpRequest.builder()
+                          .put("/{index}/_mapping/{type}")
+                          .pathParam("index", index)
+                          .pathParam("type", type)
+                          .content(MediaType.JSON, content)
+                          .build();
+    }
+}
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/V6RequestFactory.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/V6RequestFactory.java
new file mode 100644
index 0000000..aa50d43
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/V6RequestFactory.java
@@ -0,0 +1,67 @@
+/*
+ * 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.library.elasticsearch.requests.factory.v6;
+
+import org.apache.skywalking.library.elasticsearch.requests.factory.AliasFactory;
+import org.apache.skywalking.library.elasticsearch.requests.factory.BulkFactory;
+import org.apache.skywalking.library.elasticsearch.requests.factory.Codec;
+import org.apache.skywalking.library.elasticsearch.requests.factory.DocumentFactory;
+import org.apache.skywalking.library.elasticsearch.requests.factory.IndexFactory;
+import org.apache.skywalking.library.elasticsearch.requests.factory.RequestFactory;
+import org.apache.skywalking.library.elasticsearch.requests.factory.SearchFactory;
+import org.apache.skywalking.library.elasticsearch.requests.factory.TemplateFactory;
+import org.apache.skywalking.library.elasticsearch.requests.factory.v6.codec.V6Codec;
+
+public final class V6RequestFactory implements RequestFactory {
+    public static final RequestFactory INSTANCE = new V6RequestFactory();
+
+    @Override
+    public TemplateFactory template() {
+        return V6TemplateFactory.INSTANCE;
+    }
+
+    @Override
+    public IndexFactory index() {
+        return V6IndexFactory.INSTANCE;
+    }
+
+    @Override
+    public AliasFactory alias() {
+        return V6AliasFactory.INSTANCE;
+    }
+
+    @Override
+    public DocumentFactory document() {
+        return V6DocumentFactory.INSTANCE;
+    }
+
+    @Override
+    public SearchFactory search() {
+        return V6SearchFactory.INSTANCE;
+    }
+
+    @Override
+    public BulkFactory bulk() {
+        return V6BulkFactory.INSTANCE;
+    }
+
+    @Override
+    public Codec codec() {
+        return V6Codec.INSTANCE;
+    }
+}
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/V6SearchFactory.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/V6SearchFactory.java
new file mode 100644
index 0000000..2decf2a
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/V6SearchFactory.java
@@ -0,0 +1,48 @@
+/*
+ * 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.library.elasticsearch.requests.factory.v6;
+
+import com.linecorp.armeria.common.HttpRequest;
+import com.linecorp.armeria.common.HttpRequestBuilder;
+import com.linecorp.armeria.common.MediaType;
+import lombok.SneakyThrows;
+import org.apache.skywalking.library.elasticsearch.requests.factory.SearchFactory;
+import org.apache.skywalking.library.elasticsearch.requests.factory.v6.codec.V6Codec;
+import org.apache.skywalking.library.elasticsearch.requests.search.Search;
+
+final class V6SearchFactory implements SearchFactory {
+    static final SearchFactory INSTANCE = new V6SearchFactory();
+
+    @SneakyThrows
+    @Override
+    public HttpRequest search(Search search, String... indices) {
+        final HttpRequestBuilder builder = HttpRequest.builder();
+
+        if (indices == null || indices.length == 0) {
+            builder.get("/_search");
+        } else {
+            builder.get("/{indices}/_search")
+                   .pathParam("indices", String.join(",", indices));
+        }
+
+        final byte[] content = V6Codec.MAPPER.writeValueAsBytes(search);
+
+        return builder.content(MediaType.JSON, content)
+                      .build();
+    }
+}
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/V6TemplateFactory.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/V6TemplateFactory.java
new file mode 100644
index 0000000..744534f
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/V6TemplateFactory.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.library.elasticsearch.requests.factory.v6;
+
+import com.google.common.collect.ImmutableMap;
+import com.linecorp.armeria.common.HttpRequest;
+import com.linecorp.armeria.common.MediaType;
+import java.util.Collections;
+import java.util.Map;
+import lombok.SneakyThrows;
+import org.apache.skywalking.library.elasticsearch.requests.factory.TemplateFactory;
+import org.apache.skywalking.library.elasticsearch.requests.factory.v6.codec.V6Codec;
+
+final class V6TemplateFactory implements TemplateFactory {
+    static final TemplateFactory INSTANCE = new V6TemplateFactory();
+
+    @Override
+    public HttpRequest exists(String name) {
+        return HttpRequest.builder()
+                          .get("/_template/{name}")
+                          .pathParam("name", name)
+                          .build();
+    }
+
+    @Override
+    public HttpRequest get(final String name) {
+        return HttpRequest.builder()
+                          .get("/_template/{name}")
+                          .pathParam("name", name)
+                          .build();
+    }
+
+    @Override
+    public HttpRequest delete(final String name) {
+        return HttpRequest.builder()
+                          .delete("/_template/{name}")
+                          .pathParam("name", name)
+                          .build();
+    }
+
+    @SneakyThrows
+    @Override
+    public HttpRequest createOrUpdate(String name, Map<String, Object> settings,
+                                      Map<String, Object> mapping, int order) {
+        final String[] patterns = new String[] {name + "-*"};
+        final Map<String, Object> aliases = ImmutableMap.of(name, Collections.emptyMap());
+        final Map<String, Object> template =
+            ImmutableMap.<String, Object>builder()
+                        .put("index_patterns", patterns)
+                        .put("aliases", aliases)
+                        .put("settings", settings)
+                        .put("mappings", mapping)
+                        .put("order", order)
+                        .build();
+
+        final byte[] content = V6Codec.MAPPER.writeValueAsBytes(template);
+
+        return HttpRequest.builder()
+                          .put("/_template/{name}")
+                          .pathParam("name", name)
+                          .content(MediaType.JSON, content)
+                          .build();
+    }
+}
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/codec/BoolQuerySerializer.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/codec/BoolQuerySerializer.java
new file mode 100644
index 0000000..3eb5860
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/codec/BoolQuerySerializer.java
@@ -0,0 +1,68 @@
+/*
+ * 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.library.elasticsearch.requests.factory.v6.codec;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonSerializer;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import java.io.IOException;
+import java.util.List;
+import org.apache.skywalking.library.elasticsearch.requests.search.BoolQuery;
+import org.apache.skywalking.library.elasticsearch.requests.search.Query;
+
+public final class BoolQuerySerializer extends JsonSerializer<BoolQuery> {
+    static final String NAME = "bool";
+    static final String MUST = "must";
+    static final String MUST_NOT = "must_not";
+    static final String SHOULD = "should";
+    static final String SHOULD_NOT = "should_not";
+
+    @Override
+    public void serialize(final BoolQuery value, final JsonGenerator gen,
+                          final SerializerProvider provider) throws IOException {
+        gen.writeStartObject();
+        {
+            gen.writeFieldName(NAME);
+            gen.writeStartObject();
+            {
+                writeArray(gen, MUST, value.getMust());
+                writeArray(gen, MUST_NOT, value.getMustNot());
+                writeArray(gen, SHOULD, value.getShould());
+                writeArray(gen, SHOULD_NOT, value.getShouldNot());
+            }
+            gen.writeEndObject();
+        }
+        gen.writeEndObject();
+    }
+
+    private void writeArray(final JsonGenerator gen, final String name,
+                            final List<Query> array) throws IOException {
+        if (array == null) {
+            return;
+        }
+
+        gen.writeFieldName(name);
+        gen.writeStartArray();
+        {
+            for (final Query query : array) {
+                gen.writeObject(query);
+            }
+        }
+        gen.writeEndArray();
+    }
+}
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/codec/IndexRequestSerializer.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/codec/IndexRequestSerializer.java
new file mode 100644
index 0000000..ff24775
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/codec/IndexRequestSerializer.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.library.elasticsearch.requests.factory.v6.codec;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.io.SerializedString;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import com.fasterxml.jackson.databind.ser.std.StdSerializer;
+import java.io.IOException;
+import org.apache.skywalking.library.elasticsearch.requests.IndexRequest;
+
+final class IndexRequestSerializer extends StdSerializer<IndexRequest> {
+    IndexRequestSerializer() {
+        super(IndexRequest.class);
+    }
+
+    @Override
+    public void serialize(final IndexRequest value, final JsonGenerator gen,
+                          final SerializerProvider provider) throws IOException {
+        gen.setRootValueSeparator(new SerializedString("\n"));
+
+        gen.writeStartObject();
+        {
+            gen.writeFieldName("index");
+            gen.writeStartObject();
+            {
+                gen.writeStringField("_index", value.getIndex());
+                gen.writeStringField("_type", value.getType());
+                gen.writeStringField("_id", value.getId());
+            }
+            gen.writeEndObject();
+        }
+        gen.writeEndObject();
+
+        gen.writeObject(value.getDoc());
+    }
+}
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/codec/RangeQuerySerializer.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/codec/RangeQuerySerializer.java
new file mode 100644
index 0000000..dffc76c
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/codec/RangeQuerySerializer.java
@@ -0,0 +1,62 @@
+/*
+ * 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.library.elasticsearch.requests.factory.v6.codec;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonSerializer;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import java.io.IOException;
+import org.apache.skywalking.library.elasticsearch.requests.search.RangeQuery;
+
+public final class RangeQuerySerializer extends JsonSerializer<RangeQuery> {
+    static final String NAME = "range";
+
+    @Override
+    public void serialize(final RangeQuery value, final JsonGenerator gen,
+                          final SerializerProvider provider) throws IOException {
+        gen.writeStartObject();
+        {
+            gen.writeFieldName(NAME);
+            gen.writeStartObject();
+            {
+                gen.writeFieldName(value.getName());
+                gen.writeStartObject();
+
+                if (value.getGte() != null) {
+                    provider.defaultSerializeField("gte", value.getGte(), gen);
+                }
+                if (value.getLte() != null) {
+                    provider.defaultSerializeField("lte", value.getLte(), gen);
+                }
+                if (value.getGt() != null) {
+                    provider.defaultSerializeField("gt", value.getGt(), gen);
+                }
+                if (value.getLt() != null) {
+                    provider.defaultSerializeField("lt", value.getLt(), gen);
+                }
+                if (value.getBoost() != null) {
+                    provider.defaultSerializeField("boost", value.getBoost(), gen);
+                }
+
+                gen.writeEndObject();
+            }
+            gen.writeEndObject();
+        }
+        gen.writeEndObject();
+    }
+}
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/codec/TermSerializer.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/codec/TermSerializer.java
new file mode 100644
index 0000000..3de2ffa
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/codec/TermSerializer.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.library.elasticsearch.requests.factory.v6.codec;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonSerializer;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import java.io.IOException;
+import org.apache.skywalking.library.elasticsearch.requests.search.TermQuery;
+
+public final class TermSerializer extends JsonSerializer<TermQuery> {
+    static final String NAME = "term";
+
+    @Override
+    public void serialize(final TermQuery term, final JsonGenerator gen,
+                          final SerializerProvider provider) throws IOException {
+        gen.writeStartObject();
+        {
+            gen.writeFieldName(NAME);
+            gen.writeStartObject();
+            {
+                gen.writeFieldName(term.getName());
+                gen.writeObject(term.getValue());
+            }
+            gen.writeEndObject();
+        }
+        gen.writeEndObject();
+    }
+}
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/codec/UpdateRequestSerializer.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/codec/UpdateRequestSerializer.java
new file mode 100644
index 0000000..3ed6a9f
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/codec/UpdateRequestSerializer.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.library.elasticsearch.requests.factory.v6.codec;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.io.SerializedString;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import com.fasterxml.jackson.databind.ser.std.StdSerializer;
+import java.io.IOException;
+import org.apache.skywalking.library.elasticsearch.requests.UpdateRequest;
+
+final class UpdateRequestSerializer extends StdSerializer<UpdateRequest> {
+    UpdateRequestSerializer() {
+        super(UpdateRequest.class);
+    }
+
+    @Override
+    public void serialize(final UpdateRequest value, final JsonGenerator gen,
+                          final SerializerProvider provider) throws IOException {
+        gen.setRootValueSeparator(new SerializedString("\n"));
+
+        gen.writeStartObject();
+        {
+            gen.writeFieldName("update");
+            gen.writeStartObject();
+            {
+                gen.writeStringField("_index", value.getIndex());
+                gen.writeStringField("_type", value.getType());
+                gen.writeStringField("_id", value.getId());
+            }
+            gen.writeEndObject();
+        }
+
+        gen.writeEndObject();
+
+        gen.writeStartObject();
+        {
+            gen.writeFieldName("doc");
+            gen.writeObject(value.getDoc());
+        }
+        gen.writeEndObject();
+    }
+}
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/codec/V6Codec.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/codec/V6Codec.java
new file mode 100644
index 0000000..19f4ccd
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/factory/v6/codec/V6Codec.java
@@ -0,0 +1,82 @@
+/*
+ * 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.library.elasticsearch.requests.factory.v6.codec;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import java.io.InputStream;
+import org.apache.skywalking.library.elasticsearch.requests.IndexRequest;
+import org.apache.skywalking.library.elasticsearch.requests.UpdateRequest;
+import org.apache.skywalking.library.elasticsearch.requests.factory.Codec;
+import org.apache.skywalking.library.elasticsearch.requests.search.BoolQuery;
+import org.apache.skywalking.library.elasticsearch.requests.search.RangeQuery;
+import org.apache.skywalking.library.elasticsearch.requests.search.TermQuery;
+
+public final class V6Codec implements Codec {
+    public static final Codec INSTANCE = new V6Codec();
+
+    public static final ObjectMapper MAPPER = new ObjectMapper()
+        .setSerializationInclusion(JsonInclude.Include.NON_NULL)
+        // We added some serializers here and some in their item classes as annotation (e.g.
+        // org.apache.skywalking.library.elasticsearch.requests.search.Sorts),
+        // the basic idea is, if the item class is very basic and are the same serialization method
+        // in both 6.x and 7.x, we set the serializer in their item class as annotation to make it
+        // shared by 6.x and 7.x, without duplicating the serializer codes, otherwise, we add
+        // serializers for each version explicitly in the object mapper.
+        // The 2 methods to add serializers can be changed if some day the basic serializer cannot
+        // be shared between newer versions of ElasticSearch or vice versa.
+        .registerModule(
+            new SimpleModule()
+                .addSerializer(
+                    IndexRequest.class,
+                    new IndexRequestSerializer()
+                )
+                .addSerializer(
+                    UpdateRequest.class,
+                    new UpdateRequestSerializer()
+                )
+                .addSerializer(
+                    RangeQuery.class,
+                    new RangeQuerySerializer()
+                )
+                .addSerializer(
+                    TermQuery.class,
+                    new TermSerializer()
+                )
+                .addSerializer(
+                    BoolQuery.class,
+                    new BoolQuerySerializer()
+                )
+        )
+        .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+
+    @Override
+    public ByteBuf encode(final Object request) throws Exception {
+        return Unpooled.wrappedBuffer(MAPPER.writeValueAsBytes(request));
+    }
+
+    @Override
+    public <T> T decode(final InputStream inputStream,
+                        final Class<T> type) throws Exception {
+        return MAPPER.readValue(inputStream, type);
+    }
+}
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/BoolQuery.java
similarity index 61%
copy from oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
copy to oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/BoolQuery.java
index ebad19e..71dc2ae 100644
--- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/BoolQuery.java
@@ -13,27 +13,20 @@
  * 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.core.storage.type;
-
-/**
- * StorageDataComplexObject implementation supports String-Object interconversion.
- */
-public interface StorageDataComplexObject<T> {
-    /**
-     * @return string representing this object.
-     */
-    String toStorageData();
+package org.apache.skywalking.library.elasticsearch.requests.search;
 
-    /**
-     * Initialize this object based on the given string data.
-     */
-    void toObject(String data);
+import com.google.common.collect.ImmutableList;
+import lombok.AccessLevel;
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
 
-    /**
-     * Initialize the object based on the given source.
-     */
-    void copyFrom(T source);
+@Getter
+@RequiredArgsConstructor(access = AccessLevel.PACKAGE)
+public final class BoolQuery extends Query {
+    private final ImmutableList<Query> must;
+    private final ImmutableList<Query> mustNot;
+    private final ImmutableList<Query> should;
+    private final ImmutableList<Query> shouldNot;
 }
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/BoolQueryBuilder.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/BoolQueryBuilder.java
new file mode 100644
index 0000000..aacafaa
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/BoolQueryBuilder.java
@@ -0,0 +1,133 @@
+/*
+ * 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.library.elasticsearch.requests.search;
+
+import com.google.common.collect.ImmutableList;
+
+import static java.util.Objects.requireNonNull;
+
+public final class BoolQueryBuilder implements QueryBuilder {
+    private ImmutableList.Builder<Query> must;
+    private ImmutableList.Builder<Query> mustNot;
+    private ImmutableList.Builder<Query> should;
+    private ImmutableList.Builder<Query> shouldNot;
+
+    BoolQueryBuilder() {
+    }
+
+    public BoolQueryBuilder must(Query query) {
+        requireNonNull(query, "query");
+        must().add(query);
+        return this;
+    }
+
+    public BoolQueryBuilder must(QueryBuilder queryBuilder) {
+        requireNonNull(queryBuilder, "queryBuilder");
+        return must(queryBuilder.build());
+    }
+
+    public BoolQueryBuilder mustNot(Query query) {
+        requireNonNull(query, "query");
+        mustNot().add(query);
+        return this;
+    }
+
+    public BoolQueryBuilder mustNot(QueryBuilder queryBuilder) {
+        requireNonNull(queryBuilder, "queryBuilder");
+        return mustNot(queryBuilder.build());
+    }
+
+    public BoolQueryBuilder should(Query query) {
+        requireNonNull(query, "query");
+        should().add(query);
+        return this;
+    }
+
+    public BoolQueryBuilder should(QueryBuilder queryBuilder) {
+        requireNonNull(queryBuilder, "queryBuilder");
+        return should(queryBuilder.build());
+    }
+
+    public BoolQueryBuilder shouldNot(Query query) {
+        requireNonNull(query, "query");
+        shouldNot().add(query);
+        return this;
+    }
+
+    public BoolQueryBuilder shouldNot(QueryBuilder queryBuilder) {
+        requireNonNull(queryBuilder, "queryBuilder");
+        return shouldNot(queryBuilder.build());
+    }
+
+    private ImmutableList.Builder<Query> must() {
+        if (must == null) {
+            must = ImmutableList.builder();
+        }
+        return must;
+    }
+
+    private ImmutableList.Builder<Query> mustNot() {
+        if (mustNot == null) {
+            mustNot = ImmutableList.builder();
+        }
+        return mustNot;
+    }
+
+    private ImmutableList.Builder<Query> should() {
+        if (should == null) {
+            should = ImmutableList.builder();
+        }
+        return should;
+    }
+
+    private ImmutableList.Builder<Query> shouldNot() {
+        if (shouldNot == null) {
+            shouldNot = ImmutableList.builder();
+        }
+        return shouldNot;
+    }
+
+    @Override
+    public Query build() {
+        final ImmutableList<Query> must;
+        if (this.must == null) {
+            must = null;
+        } else {
+            must = this.must.build();
+        }
+        final ImmutableList<Query> should;
+        if (this.should == null) {
+            should = null;
+        } else {
+            should = this.should.build();
+        }
+        final ImmutableList<Query> mustNot;
+        if (this.mustNot == null) {
+            mustNot = null;
+        } else {
+            mustNot = this.mustNot.build();
+        }
+        final ImmutableList<Query> shouldNot;
+        if (this.shouldNot == null) {
+            shouldNot = null;
+        } else {
+            shouldNot = this.shouldNot.build();
+        }
+        return new BoolQuery(must, mustNot, should, shouldNot);
+    }
+}
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/IdsQuery.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/IdsQuery.java
new file mode 100644
index 0000000..103834d
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/IdsQuery.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.library.elasticsearch.requests.search;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import com.fasterxml.jackson.databind.annotation.JsonSerialize;
+import com.fasterxml.jackson.databind.ser.std.StdSerializer;
+import com.google.common.collect.ImmutableList;
+import java.io.IOException;
+import lombok.AccessLevel;
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+
+@Getter
+@JsonSerialize(using = IdsQuery.Serializer.class)
+@RequiredArgsConstructor(access = AccessLevel.PACKAGE)
+public final class IdsQuery extends Query {
+    private static final String NAME = "ids";
+
+    private final ImmutableList<String> ids;
+
+    static class Serializer extends StdSerializer<IdsQuery> {
+        protected Serializer() {
+            super(IdsQuery.class);
+        }
+
+        @Override
+        public void serialize(final IdsQuery value, final JsonGenerator gen,
+                              final SerializerProvider provider)
+            throws IOException {
+            gen.writeFieldName(NAME);
+            gen.writeStartObject();
+            gen.writeObjectField("values", ids());
+            gen.writeEndObject();
+        }
+    }
+}
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/MatchPhaseQuery.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/MatchPhaseQuery.java
new file mode 100644
index 0000000..01853fb
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/MatchPhaseQuery.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.library.elasticsearch.requests.search;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import com.fasterxml.jackson.databind.annotation.JsonSerialize;
+import com.fasterxml.jackson.databind.ser.std.StdSerializer;
+import java.io.IOException;
+import lombok.AccessLevel;
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+
+@Getter
+@JsonSerialize(using = MatchPhaseQuery.Serializer.class)
+@RequiredArgsConstructor(access = AccessLevel.PACKAGE)
+public final class MatchPhaseQuery extends Query {
+    private final String name;
+    private final String text;
+
+    static class Serializer extends StdSerializer<MatchPhaseQuery> {
+        protected Serializer() {
+            super(MatchPhaseQuery.class);
+        }
+
+        @Override
+        public void serialize(final MatchPhaseQuery value, final JsonGenerator gen,
+                              final SerializerProvider provider)
+            throws IOException {
+            gen.writeFieldName("match_phrase");
+            gen.writeStartObject();
+            {
+                gen.writeStringField(value.getName(), value.getText());
+            }
+            gen.writeEndObject();
+        }
+    }
+}
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/MatchQuery.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/MatchQuery.java
new file mode 100644
index 0000000..48cb302
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/MatchQuery.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.library.elasticsearch.requests.search;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import com.fasterxml.jackson.databind.annotation.JsonSerialize;
+import com.fasterxml.jackson.databind.ser.std.StdSerializer;
+import java.io.IOException;
+import lombok.AccessLevel;
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+
+@Getter
+@JsonSerialize(using = MatchQuery.Serializer.class)
+@RequiredArgsConstructor(access = AccessLevel.PACKAGE)
+public final class MatchQuery extends Query {
+    private final String name;
+    private final String text;
+
+    static class Serializer extends StdSerializer<MatchQuery> {
+        protected Serializer() {
+            super(MatchQuery.class);
+        }
+
+        @Override
+        public void serialize(final MatchQuery value, final JsonGenerator gen,
+                              final SerializerProvider provider)
+            throws IOException {
+            gen.writeFieldName("match");
+            gen.writeStartObject();
+            {
+                gen.writeStringField(value.getName(), value.getText());
+            }
+            gen.writeEndObject();
+        }
+    }
+}
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/Query.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/Query.java
new file mode 100644
index 0000000..5ec0d40
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/Query.java
@@ -0,0 +1,80 @@
+/*
+ * 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.library.elasticsearch.requests.search;
+
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableList;
+import java.util.Arrays;
+import java.util.List;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * Represents criteria when matching documents in ElasticSearch.
+ */
+public abstract class Query {
+    public static RangeQueryBuilder range(String name) {
+        checkArgument(!Strings.isNullOrEmpty(name), "name cannot be blank");
+        return new RangeQueryBuilder(name);
+    }
+
+    public static TermQuery term(String name, Object value) {
+        checkArgument(!Strings.isNullOrEmpty(name), "name cannot be blank");
+        requireNonNull(value, "value");
+        return new TermQuery(name, value);
+    }
+
+    public static TermsQuery terms(String name, List<?> values) {
+        checkArgument(!Strings.isNullOrEmpty(name), "name cannot be blank");
+        requireNonNull(values, "values");
+        return new TermsQuery(name, values);
+    }
+
+    public static MatchQuery match(String name, String text) {
+        checkArgument(!Strings.isNullOrEmpty(name), "name cannot be blank");
+        checkArgument(!Strings.isNullOrEmpty(text), "text cannot be blank");
+        return new MatchQuery(name, text);
+    }
+
+    public static MatchQuery match(String name, Object value) {
+        requireNonNull(value, "value");
+        return match(name, value.toString());
+    }
+
+    public static MatchPhaseQuery matchPhrase(String name, String text) {
+        checkArgument(!Strings.isNullOrEmpty(name), "name cannot be blank");
+        checkArgument(!Strings.isNullOrEmpty(text), "text cannot be blank");
+        return new MatchPhaseQuery(name, text);
+    }
+
+    public static IdsQuery ids(String... ids) {
+        requireNonNull(ids, "ids");
+        checkArgument(ids.length > 0, "ids cannot be empty");
+        return ids(Arrays.asList(ids));
+    }
+
+    public static IdsQuery ids(Iterable<String> ids) {
+        requireNonNull(ids, "ids");
+        return new IdsQuery(ImmutableList.<String>builder().addAll(ids).build());
+    }
+
+    public static BoolQueryBuilder bool() {
+        return new BoolQueryBuilder();
+    }
+}
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/QueryBuilder.java
similarity index 61%
copy from oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
copy to oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/QueryBuilder.java
index ebad19e..8b7a948 100644
--- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/QueryBuilder.java
@@ -13,27 +13,10 @@
  * 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.core.storage.type;
-
-/**
- * StorageDataComplexObject implementation supports String-Object interconversion.
  */
-public interface StorageDataComplexObject<T> {
-    /**
-     * @return string representing this object.
-     */
-    String toStorageData();
 
-    /**
-     * Initialize this object based on the given string data.
-     */
-    void toObject(String data);
+package org.apache.skywalking.library.elasticsearch.requests.search;
 
-    /**
-     * Initialize the object based on the given source.
-     */
-    void copyFrom(T source);
+public interface QueryBuilder {
+    Query build();
 }
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/RangeQuery.java
similarity index 61%
copy from oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
copy to oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/RangeQuery.java
index ebad19e..403f6f8 100644
--- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/RangeQuery.java
@@ -13,27 +13,21 @@
  * 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.core.storage.type;
-
-/**
- * StorageDataComplexObject implementation supports String-Object interconversion.
- */
-public interface StorageDataComplexObject<T> {
-    /**
-     * @return string representing this object.
-     */
-    String toStorageData();
+package org.apache.skywalking.library.elasticsearch.requests.search;
 
-    /**
-     * Initialize this object based on the given string data.
-     */
-    void toObject(String data);
+import lombok.AccessLevel;
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
 
-    /**
-     * Initialize the object based on the given source.
-     */
-    void copyFrom(T source);
+@Getter
+@RequiredArgsConstructor(access = AccessLevel.PACKAGE)
+public final class RangeQuery extends Query {
+    private final String name;
+    private final Object gte;
+    private final Object gt;
+    private final Object lte;
+    private final Object lt;
+    private final Double boost;
 }
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/RangeQueryBuilder.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/RangeQueryBuilder.java
new file mode 100644
index 0000000..1430862
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/RangeQueryBuilder.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.library.elasticsearch.requests.search;
+
+import com.google.common.base.Strings;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
+
+public final class RangeQueryBuilder implements QueryBuilder {
+    private final String name;
+    private Object gte;
+    private Object gt;
+    private Object lte;
+    private Object lt;
+    private Double boost;
+
+    RangeQueryBuilder(String name) {
+        checkArgument(!Strings.isNullOrEmpty(name), "name cannot be null or empty");
+
+        this.name = name;
+    }
+
+    public RangeQueryBuilder gte(Object gte) {
+        this.gte = requireNonNull(gte, "gte");
+        return this;
+    }
+
+    public RangeQueryBuilder gt(Object gt) {
+        this.gt = requireNonNull(gt, "gt");
+        return this;
+    }
+
+    public RangeQueryBuilder lte(Object lte) {
+        this.lte = requireNonNull(lte, "lte");
+        return this;
+    }
+
+    public RangeQueryBuilder lt(Object lt) {
+        this.lt = requireNonNull(lt, "lt");
+        return this;
+    }
+
+    public RangeQueryBuilder boost(Double boost) {
+        requireNonNull(boost, "boost");
+
+        this.boost = boost;
+        return this;
+    }
+
+    @Override
+    public Query build() {
+        return new RangeQuery(name, gte, gt, lte, lt, boost);
+    }
+}
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/Search.java
similarity index 50%
copy from oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
copy to oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/Search.java
index ebad19e..7b0aec8 100644
--- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/Search.java
@@ -13,27 +13,31 @@
  * 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.core.storage.type;
+package org.apache.skywalking.library.elasticsearch.requests.search;
+
+import com.google.common.collect.ImmutableMap;
+import lombok.AccessLevel;
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.skywalking.library.elasticsearch.requests.search.aggregation.Aggregation;
 
 /**
- * StorageDataComplexObject implementation supports String-Object interconversion.
+ * Represents the criteria when searching documents in ElasticSearch.
+ *
+ * @see SearchBuilder to build an immutable instance of this class.
  */
-public interface StorageDataComplexObject<T> {
-    /**
-     * @return string representing this object.
-     */
-    String toStorageData();
-
-    /**
-     * Initialize this object based on the given string data.
-     */
-    void toObject(String data);
+@Getter
+@RequiredArgsConstructor(access = AccessLevel.PACKAGE)
+public final class Search {
+    private final Integer from;
+    private final Integer size;
+    private final Query query;
+    private final Sorts sort;
+    private final ImmutableMap<String, Aggregation> aggregations;
 
-    /**
-     * Initialize the object based on the given source.
-     */
-    void copyFrom(T source);
+    public static SearchBuilder builder() {
+        return new SearchBuilder();
+    }
 }
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/SearchBuilder.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/SearchBuilder.java
new file mode 100644
index 0000000..3705f5b
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/SearchBuilder.java
@@ -0,0 +1,122 @@
+/*
+ * 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.library.elasticsearch.requests.search;
+
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.skywalking.library.elasticsearch.requests.search.aggregation.Aggregation;
+import org.apache.skywalking.library.elasticsearch.requests.search.aggregation.AggregationBuilder;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
+
+public final class SearchBuilder {
+    private String[] indices; // TODO: move indices parameter here
+
+    private Integer from;
+    private Integer size;
+    private Query query;
+    private ImmutableList.Builder<Sort> sort;
+    private ImmutableMap.Builder<String, Aggregation> aggregations;
+
+    SearchBuilder() {
+    }
+
+    public SearchBuilder index(String... indices) {
+        requireNonNull(indices, "indices");
+        this.indices = indices;
+        return this;
+    }
+
+    public SearchBuilder from(Integer from) {
+        requireNonNull(from, "from");
+        checkArgument(from > 0, "from must be positive");
+        this.from = from;
+        return this;
+    }
+
+    public SearchBuilder size(Integer size) {
+        requireNonNull(size, "size");
+        checkArgument(size > 0, "size must be positive");
+        this.size = size;
+        return this;
+    }
+
+    public SearchBuilder sort(String by, Sort.Order order) {
+        checkArgument(!Strings.isNullOrEmpty(by), "by must be positive");
+        requireNonNull(order, "order");
+        sort().add(new Sort(by, order));
+        return this;
+    }
+
+    public SearchBuilder query(Query query) {
+        requireNonNull(query, "query");
+        this.query = query;
+        return this;
+    }
+
+    public SearchBuilder query(QueryBuilder queryBuilder) {
+        return query(queryBuilder.build());
+    }
+
+    public SearchBuilder aggregation(Aggregation aggregation) {
+        requireNonNull(aggregation, "aggregation");
+        aggregations().put(aggregation.name(), aggregation);
+        return this;
+    }
+
+    public SearchBuilder aggregation(AggregationBuilder builder) {
+        requireNonNull(builder, "builder");
+        return aggregation(builder.build());
+    }
+
+    public Search build() {
+        final Sorts sorts;
+        if (sort == null) {
+            sorts = null;
+        } else {
+            sorts = new Sorts(sort.build());
+        }
+
+        final ImmutableMap<String, Aggregation> aggregations;
+        if (this.aggregations == null) {
+            aggregations = null;
+        } else {
+            aggregations = aggregations().build();
+        }
+
+        return new Search(
+            from, size, query, sorts, aggregations
+        );
+    }
+
+    private ImmutableList.Builder<Sort> sort() {
+        if (sort == null) {
+            sort = ImmutableList.builder();
+        }
+        return sort;
+    }
+
+    private ImmutableMap.Builder<String, Aggregation> aggregations() {
+        if (aggregations == null) {
+            aggregations = ImmutableMap.builder();
+        }
+        return aggregations;
+    }
+}
diff --git a/oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ElasticSearchUpdateRequest.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/Sort.java
similarity index 58%
copy from oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ElasticSearchUpdateRequest.java
copy to oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/Sort.java
index 2663856..e99bf39 100644
--- a/oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ElasticSearchUpdateRequest.java
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/Sort.java
@@ -15,20 +15,30 @@
  * limitations under the License.
  */
 
-package org.apache.skywalking.oap.server.library.client.elasticsearch;
+package org.apache.skywalking.library.elasticsearch.requests.search;
 
-import org.elasticsearch.action.update.UpdateRequest;
-import org.elasticsearch.common.xcontent.XContentBuilder;
+import lombok.AccessLevel;
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
 
-public class ElasticSearchUpdateRequest extends UpdateRequest implements org.apache.skywalking.oap.server.library.client.request.UpdateRequest {
+@Getter
+@RequiredArgsConstructor(access = AccessLevel.PACKAGE)
+public final class Sort {
+    private final String name;
+    private final Order order;
 
-    public ElasticSearchUpdateRequest(String index, String type, String id) {
-        super(index, type, id);
-    }
+    public enum Order {
+        ASC("asc"), DESC("desc");
+
+        final String value;
+
+        Order(final String value) {
+            this.value = value;
+        }
 
-    @Override
-    public ElasticSearchUpdateRequest doc(XContentBuilder source) {
-        super.doc(source);
-        return this;
+        @Override
+        public String toString() {
+            return value;
+        }
     }
 }
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/Sorts.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/Sorts.java
new file mode 100644
index 0000000..f1612d6
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/Sorts.java
@@ -0,0 +1,74 @@
+/*
+ * 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.library.elasticsearch.requests.search;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import com.fasterxml.jackson.databind.annotation.JsonSerialize;
+import com.fasterxml.jackson.databind.ser.std.StdSerializer;
+import com.google.common.collect.ImmutableList;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Iterator;
+import lombok.AccessLevel;
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+
+@Getter
+@JsonSerialize(using = Sorts.Serializer.class)
+@RequiredArgsConstructor(access = AccessLevel.PACKAGE)
+final class Sorts implements Iterable<Sort> {
+    private final ImmutableList<Sort> sorts;
+
+    @Override
+    public Iterator<Sort> iterator() {
+        if (sorts == null) {
+            return Collections.emptyIterator();
+        }
+        return sorts.iterator();
+    }
+
+    static class Serializer extends StdSerializer<Sorts> {
+        protected Serializer() {
+            super(Sorts.class);
+        }
+
+        @Override
+        public void serialize(final Sorts value, final JsonGenerator gen,
+                              final SerializerProvider provider)
+            throws IOException {
+
+            gen.writeStartArray();
+            {
+                for (final Sort sort : value) {
+                    gen.writeStartObject();
+                    {
+                        gen.writeFieldName(sort.getName());
+                        gen.writeStartObject();
+                        {
+                            gen.writeStringField("order", sort.getOrder().toString());
+                        }
+                        gen.writeEndObject();
+                    }
+                    gen.writeEndObject();
+                }
+            }
+            gen.writeEndArray();
+        }
+    }
+}
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/TermQuery.java
similarity index 61%
copy from oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
copy to oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/TermQuery.java
index ebad19e..236c6e5 100644
--- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/TermQuery.java
@@ -13,27 +13,17 @@
  * 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.core.storage.type;
-
-/**
- * StorageDataComplexObject implementation supports String-Object interconversion.
- */
-public interface StorageDataComplexObject<T> {
-    /**
-     * @return string representing this object.
-     */
-    String toStorageData();
+package org.apache.skywalking.library.elasticsearch.requests.search;
 
-    /**
-     * Initialize this object based on the given string data.
-     */
-    void toObject(String data);
+import lombok.AccessLevel;
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
 
-    /**
-     * Initialize the object based on the given source.
-     */
-    void copyFrom(T source);
+@Getter
+@RequiredArgsConstructor(access = AccessLevel.PACKAGE)
+public final class TermQuery extends Query {
+    private final String name;
+    private final Object value;
 }
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/TermsQuery.java
similarity index 61%
copy from oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
copy to oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/TermsQuery.java
index ebad19e..f3ca634 100644
--- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/TermsQuery.java
@@ -13,27 +13,18 @@
  * 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.core.storage.type;
-
-/**
- * StorageDataComplexObject implementation supports String-Object interconversion.
- */
-public interface StorageDataComplexObject<T> {
-    /**
-     * @return string representing this object.
-     */
-    String toStorageData();
+package org.apache.skywalking.library.elasticsearch.requests.search;
 
-    /**
-     * Initialize this object based on the given string data.
-     */
-    void toObject(String data);
+import java.util.List;
+import lombok.AccessLevel;
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
 
-    /**
-     * Initialize the object based on the given source.
-     */
-    void copyFrom(T source);
+@Getter
+@RequiredArgsConstructor(access = AccessLevel.PACKAGE)
+public final class TermsQuery extends Query {
+    private final String name;
+    private final List<?> values;
 }
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/Aggregation.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/Aggregation.java
new file mode 100644
index 0000000..be1c72f
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/Aggregation.java
@@ -0,0 +1,50 @@
+/*
+ * 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.library.elasticsearch.requests.search.aggregation;
+
+import com.google.common.base.Strings;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+public abstract class Aggregation {
+    public abstract String name();
+
+    public static TermsAggregationBuilder terms(String name) {
+        return new TermsAggregationBuilder(name);
+    }
+
+    public static AvgAggregationBuilder avg(String name) {
+        checkArgument(!Strings.isNullOrEmpty(name), "name cannot be blank");
+        return new AvgAggregationBuilder(name);
+    }
+
+    public static AvgAggregationBuilder min(String name) {
+        checkArgument(!Strings.isNullOrEmpty(name), "name cannot be blank");
+        return new AvgAggregationBuilder(name);
+    }
+
+    public static MaxAggregationBuilder max(String name) {
+        checkArgument(!Strings.isNullOrEmpty(name), "name cannot be blank");
+        return new MaxAggregationBuilder(name);
+    }
+
+    public static SumAggregationBuilder sum(String name) {
+        checkArgument(!Strings.isNullOrEmpty(name), "name cannot be blank");
+        return new SumAggregationBuilder(name);
+    }
+}
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/AggregationBuilder.java
similarity index 61%
copy from oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
copy to oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/AggregationBuilder.java
index ebad19e..732a3b6 100644
--- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/AggregationBuilder.java
@@ -13,27 +13,10 @@
  * 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.core.storage.type;
-
-/**
- * StorageDataComplexObject implementation supports String-Object interconversion.
  */
-public interface StorageDataComplexObject<T> {
-    /**
-     * @return string representing this object.
-     */
-    String toStorageData();
 
-    /**
-     * Initialize this object based on the given string data.
-     */
-    void toObject(String data);
+package org.apache.skywalking.library.elasticsearch.requests.search.aggregation;
 
-    /**
-     * Initialize the object based on the given source.
-     */
-    void copyFrom(T source);
+public interface AggregationBuilder {
+    Aggregation build();
 }
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/AvgAggregation.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/AvgAggregation.java
new file mode 100644
index 0000000..94585d8
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/AvgAggregation.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.library.elasticsearch.requests.search.aggregation;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import com.fasterxml.jackson.databind.annotation.JsonSerialize;
+import com.fasterxml.jackson.databind.ser.std.StdSerializer;
+import java.io.IOException;
+import lombok.AccessLevel;
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+
+@Getter
+@RequiredArgsConstructor(access = AccessLevel.PACKAGE)
+@JsonSerialize(using = AvgAggregation.Serializer.class)
+public final class AvgAggregation extends Aggregation {
+    private final String name;
+    private final String field;
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    static class Serializer extends StdSerializer<AvgAggregation> {
+        protected Serializer() {
+            super(AvgAggregation.class);
+        }
+
+        @Override
+        public void serialize(final AvgAggregation value, final JsonGenerator gen,
+                              final SerializerProvider provider) throws IOException {
+            gen.writeStartObject();
+            {
+                gen.writeFieldName("avg");
+                gen.writeStartObject();
+                {
+                    gen.writeStringField("field", value.getField());
+                }
+                gen.writeEndObject();
+            }
+            gen.writeEndObject();
+        }
+    }
+}
diff --git a/oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ElasticSearchInsertRequest.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/AvgAggregationBuilder.java
similarity index 57%
copy from oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ElasticSearchInsertRequest.java
copy to oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/AvgAggregationBuilder.java
index 9c0655c..53d050c 100644
--- a/oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ElasticSearchInsertRequest.java
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/AvgAggregationBuilder.java
@@ -15,21 +15,29 @@
  * limitations under the License.
  */
 
-package org.apache.skywalking.oap.server.library.client.elasticsearch;
+package org.apache.skywalking.library.elasticsearch.requests.search.aggregation;
 
-import org.apache.skywalking.oap.server.library.client.request.InsertRequest;
-import org.elasticsearch.action.index.IndexRequest;
-import org.elasticsearch.common.xcontent.XContentBuilder;
+import com.google.common.base.Strings;
 
-public class ElasticSearchInsertRequest extends IndexRequest implements InsertRequest {
+import static com.google.common.base.Preconditions.checkArgument;
 
-    public ElasticSearchInsertRequest(String index, String type, String id) {
-        super(index, type, id);
+public final class AvgAggregationBuilder implements AggregationBuilder {
+    private final String name;
+
+    private String field;
+
+    AvgAggregationBuilder(String name) {
+        this.name = name;
     }
 
-    @Override
-    public ElasticSearchInsertRequest source(XContentBuilder sourceBuilder) {
-        super.source(sourceBuilder);
+    public AvgAggregationBuilder field(String field) {
+        checkArgument(!Strings.isNullOrEmpty(field), "field cannot be blank");
+        this.field = field;
         return this;
     }
+
+    @Override
+    public AvgAggregation build() {
+        return new AvgAggregation(name, field);
+    }
 }
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/BucketOrder.java
similarity index 61%
copy from oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
copy to oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/BucketOrder.java
index ebad19e..e7e23f4 100644
--- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/BucketOrder.java
@@ -13,27 +13,21 @@
  * 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.core.storage.type;
+package org.apache.skywalking.library.elasticsearch.requests.search.aggregation;
 
-/**
- * StorageDataComplexObject implementation supports String-Object interconversion.
- */
-public interface StorageDataComplexObject<T> {
-    /**
-     * @return string representing this object.
-     */
-    String toStorageData();
+import lombok.AccessLevel;
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
 
-    /**
-     * Initialize this object based on the given string data.
-     */
-    void toObject(String data);
+@Getter
+@RequiredArgsConstructor(access = AccessLevel.PACKAGE)
+public final class BucketOrder {
+    private final String path;
+    private final boolean asc;
 
-    /**
-     * Initialize the object based on the given source.
-     */
-    void copyFrom(T source);
+    public static BucketOrder aggregation(String path, boolean asc) {
+        return new BucketOrder(path, asc);
+    }
 }
diff --git a/oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ElasticSearchUpdateRequest.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/MaxAggregation.java
similarity index 59%
rename from oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ElasticSearchUpdateRequest.java
rename to oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/MaxAggregation.java
index 2663856..aff33a2 100644
--- a/oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ElasticSearchUpdateRequest.java
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/MaxAggregation.java
@@ -15,20 +15,20 @@
  * limitations under the License.
  */
 
-package org.apache.skywalking.oap.server.library.client.elasticsearch;
+package org.apache.skywalking.library.elasticsearch.requests.search.aggregation;
 
-import org.elasticsearch.action.update.UpdateRequest;
-import org.elasticsearch.common.xcontent.XContentBuilder;
+import lombok.AccessLevel;
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
 
-public class ElasticSearchUpdateRequest extends UpdateRequest implements org.apache.skywalking.oap.server.library.client.request.UpdateRequest {
-
-    public ElasticSearchUpdateRequest(String index, String type, String id) {
-        super(index, type, id);
-    }
+@Getter
+@RequiredArgsConstructor(access = AccessLevel.PACKAGE)
+public final class MaxAggregation extends Aggregation {
+    private final String name;
+    private final String field;
 
     @Override
-    public ElasticSearchUpdateRequest doc(XContentBuilder source) {
-        super.doc(source);
-        return this;
+    public String name() {
+        return name;
     }
 }
diff --git a/oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ElasticSearchInsertRequest.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/MaxAggregationBuilder.java
similarity index 57%
copy from oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ElasticSearchInsertRequest.java
copy to oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/MaxAggregationBuilder.java
index 9c0655c..8b74160 100644
--- a/oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ElasticSearchInsertRequest.java
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/MaxAggregationBuilder.java
@@ -15,21 +15,29 @@
  * limitations under the License.
  */
 
-package org.apache.skywalking.oap.server.library.client.elasticsearch;
+package org.apache.skywalking.library.elasticsearch.requests.search.aggregation;
 
-import org.apache.skywalking.oap.server.library.client.request.InsertRequest;
-import org.elasticsearch.action.index.IndexRequest;
-import org.elasticsearch.common.xcontent.XContentBuilder;
+import com.google.common.base.Strings;
 
-public class ElasticSearchInsertRequest extends IndexRequest implements InsertRequest {
+import static com.google.common.base.Preconditions.checkArgument;
 
-    public ElasticSearchInsertRequest(String index, String type, String id) {
-        super(index, type, id);
+public final class MaxAggregationBuilder implements AggregationBuilder {
+    private final String name;
+
+    private String field;
+
+    MaxAggregationBuilder(String name) {
+        this.name = name;
     }
 
-    @Override
-    public ElasticSearchInsertRequest source(XContentBuilder sourceBuilder) {
-        super.source(sourceBuilder);
+    public MaxAggregationBuilder field(String field) {
+        checkArgument(!Strings.isNullOrEmpty(field), "field cannot be blank");
+        this.field = field;
         return this;
     }
+
+    @Override
+    public MaxAggregation build() {
+        return new MaxAggregation(name, field);
+    }
 }
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/MinAggregation.java
similarity index 61%
copy from oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
copy to oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/MinAggregation.java
index ebad19e..aa90c5c 100644
--- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/MinAggregation.java
@@ -13,27 +13,22 @@
  * 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.core.storage.type;
+package org.apache.skywalking.library.elasticsearch.requests.search.aggregation;
 
-/**
- * StorageDataComplexObject implementation supports String-Object interconversion.
- */
-public interface StorageDataComplexObject<T> {
-    /**
-     * @return string representing this object.
-     */
-    String toStorageData();
+import lombok.AccessLevel;
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
 
-    /**
-     * Initialize this object based on the given string data.
-     */
-    void toObject(String data);
+@Getter
+@RequiredArgsConstructor(access = AccessLevel.PACKAGE)
+public final class MinAggregation extends Aggregation {
+    private final String name;
+    private final String field;
 
-    /**
-     * Initialize the object based on the given source.
-     */
-    void copyFrom(T source);
+    @Override
+    public String name() {
+        return name;
+    }
 }
diff --git a/oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ElasticSearchInsertRequest.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/MinAggregationBuilder.java
similarity index 57%
copy from oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ElasticSearchInsertRequest.java
copy to oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/MinAggregationBuilder.java
index 9c0655c..02e2c9e 100644
--- a/oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ElasticSearchInsertRequest.java
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/MinAggregationBuilder.java
@@ -15,21 +15,29 @@
  * limitations under the License.
  */
 
-package org.apache.skywalking.oap.server.library.client.elasticsearch;
+package org.apache.skywalking.library.elasticsearch.requests.search.aggregation;
 
-import org.apache.skywalking.oap.server.library.client.request.InsertRequest;
-import org.elasticsearch.action.index.IndexRequest;
-import org.elasticsearch.common.xcontent.XContentBuilder;
+import com.google.common.base.Strings;
 
-public class ElasticSearchInsertRequest extends IndexRequest implements InsertRequest {
+import static com.google.common.base.Preconditions.checkArgument;
 
-    public ElasticSearchInsertRequest(String index, String type, String id) {
-        super(index, type, id);
+public final class MinAggregationBuilder implements AggregationBuilder {
+    private final String name;
+
+    private String field;
+
+    MinAggregationBuilder(String name) {
+        this.name = name;
     }
 
-    @Override
-    public ElasticSearchInsertRequest source(XContentBuilder sourceBuilder) {
-        super.source(sourceBuilder);
+    public MinAggregationBuilder field(String field) {
+        checkArgument(!Strings.isNullOrEmpty(field), "field cannot be blank");
+        this.field = field;
         return this;
     }
+
+    @Override
+    public MinAggregation build() {
+        return new MinAggregation(name, field);
+    }
 }
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/SumAggregation.java
similarity index 61%
copy from oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
copy to oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/SumAggregation.java
index ebad19e..c5e8a93 100644
--- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/SumAggregation.java
@@ -13,27 +13,22 @@
  * 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.core.storage.type;
+package org.apache.skywalking.library.elasticsearch.requests.search.aggregation;
 
-/**
- * StorageDataComplexObject implementation supports String-Object interconversion.
- */
-public interface StorageDataComplexObject<T> {
-    /**
-     * @return string representing this object.
-     */
-    String toStorageData();
+import lombok.AccessLevel;
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
 
-    /**
-     * Initialize this object based on the given string data.
-     */
-    void toObject(String data);
+@Getter
+@RequiredArgsConstructor(access = AccessLevel.PACKAGE)
+public final class SumAggregation extends Aggregation {
+    private final String name;
+    private final String field;
 
-    /**
-     * Initialize the object based on the given source.
-     */
-    void copyFrom(T source);
+    @Override
+    public String name() {
+        return name;
+    }
 }
diff --git a/oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ElasticSearchInsertRequest.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/SumAggregationBuilder.java
similarity index 57%
copy from oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ElasticSearchInsertRequest.java
copy to oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/SumAggregationBuilder.java
index 9c0655c..b7a7f88 100644
--- a/oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ElasticSearchInsertRequest.java
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/SumAggregationBuilder.java
@@ -15,21 +15,29 @@
  * limitations under the License.
  */
 
-package org.apache.skywalking.oap.server.library.client.elasticsearch;
+package org.apache.skywalking.library.elasticsearch.requests.search.aggregation;
 
-import org.apache.skywalking.oap.server.library.client.request.InsertRequest;
-import org.elasticsearch.action.index.IndexRequest;
-import org.elasticsearch.common.xcontent.XContentBuilder;
+import com.google.common.base.Strings;
 
-public class ElasticSearchInsertRequest extends IndexRequest implements InsertRequest {
+import static com.google.common.base.Preconditions.checkArgument;
 
-    public ElasticSearchInsertRequest(String index, String type, String id) {
-        super(index, type, id);
+public final class SumAggregationBuilder implements AggregationBuilder {
+    private final String name;
+
+    private String field;
+
+    SumAggregationBuilder(String name) {
+        this.name = name;
     }
 
-    @Override
-    public ElasticSearchInsertRequest source(XContentBuilder sourceBuilder) {
-        super.source(sourceBuilder);
+    public SumAggregationBuilder field(String field) {
+        checkArgument(!Strings.isNullOrEmpty(field), "field cannot be blank");
+        this.field = field;
         return this;
     }
+
+    @Override
+    public MaxAggregation build() {
+        return new MaxAggregation(name, field);
+    }
 }
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/TermsAggregation.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/TermsAggregation.java
new file mode 100644
index 0000000..8fb80c3
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/TermsAggregation.java
@@ -0,0 +1,69 @@
+/*
+ * 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.library.elasticsearch.requests.search.aggregation;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import com.fasterxml.jackson.databind.annotation.JsonSerialize;
+import com.fasterxml.jackson.databind.ser.std.StdSerializer;
+import com.google.common.collect.ImmutableMap;
+import java.io.IOException;
+import lombok.AccessLevel;
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+
+@Getter
+@RequiredArgsConstructor(access = AccessLevel.PACKAGE)
+@JsonSerialize(using = TermsAggregation.Serializer.class)
+public final class TermsAggregation extends Aggregation {
+    private final String name;
+    private final String field;
+    private final BucketOrder order;
+    private final Integer size;
+    private final ImmutableMap<String, Aggregation> aggregations;
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    static class Serializer extends StdSerializer<TermsAggregation> {
+        protected Serializer() {
+            super(TermsAggregation.class);
+        }
+
+        @Override
+        public void serialize(final TermsAggregation value, final JsonGenerator gen,
+                              final SerializerProvider provider) throws IOException {
+            gen.writeStartObject();
+            {
+                gen.writeFieldName("terms");
+                gen.writeStartObject();
+                {
+                    gen.writeStringField("field", value.getField());
+                }
+                gen.writeEndObject();
+
+                if (value.getAggregations() != null && !value.getAggregations().isEmpty()) {
+                    gen.writeObjectField("aggregations", value.getAggregations());
+                }
+            }
+            gen.writeEndObject();
+        }
+    }
+}
diff --git a/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/TermsAggregationBuilder.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/TermsAggregationBuilder.java
new file mode 100644
index 0000000..3c3ccbc
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/requests/search/aggregation/TermsAggregationBuilder.java
@@ -0,0 +1,87 @@
+/*
+ * 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.library.elasticsearch.requests.search.aggregation;
+
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableMap;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
+
+public final class TermsAggregationBuilder implements AggregationBuilder {
+    private final String name;
+
+    private String field;
+    private BucketOrder order;
+    private Integer size;
+    private ImmutableMap.Builder<String, Aggregation> subAggregations;
+
+    TermsAggregationBuilder(final String name) {
+        checkArgument(!Strings.isNullOrEmpty(name), "name cannot be blank");
+        this.name = name;
+    }
+
+    public TermsAggregationBuilder field(String field) {
+        checkArgument(!Strings.isNullOrEmpty(field), "field cannot be blank");
+        this.field = field;
+        return this;
+    }
+
+    public TermsAggregationBuilder order(BucketOrder order) {
+        requireNonNull(order, "order");
+        this.order = order;
+        return this;
+    }
+
+    public TermsAggregationBuilder size(int size) {
+        checkArgument(size >= 0, "size must be >= 0");
+        this.size = size;
+        return this;
+    }
+
+    public TermsAggregationBuilder subAggregation(Aggregation subAggregation) {
+        requireNonNull(subAggregation, "subAggregation");
+        subAggregations().put(subAggregation.name(), subAggregation);
+        return this;
+    }
+
+    public TermsAggregationBuilder subAggregation(AggregationBuilder subAggregationBuilder) {
+        requireNonNull(subAggregationBuilder, "subAggregationBuilder");
+        return subAggregation(subAggregationBuilder.build());
+    }
+
+    @Override
+    public TermsAggregation build() {
+        ImmutableMap<String, Aggregation> subAggregations;
+        if (this.subAggregations == null) {
+            subAggregations = null;
+        } else {
+            subAggregations = this.subAggregations.build();
+        }
+        return new TermsAggregation(
+            name, field, order, size, subAggregations
+        );
+    }
+
+    private ImmutableMap.Builder<String, Aggregation> subAggregations() {
+        if (subAggregations == null) {
+            subAggregations = ImmutableMap.builder();
+        }
+        return subAggregations;
+    }
+}
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/response/Document.java
similarity index 61%
copy from oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
copy to oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/response/Document.java
index ebad19e..89e0565 100644
--- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/response/Document.java
@@ -13,27 +13,21 @@
  * 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.core.storage.type;
+package org.apache.skywalking.library.elasticsearch.response;
 
-/**
- * StorageDataComplexObject implementation supports String-Object interconversion.
- */
-public interface StorageDataComplexObject<T> {
-    /**
-     * @return string representing this object.
-     */
-    String toStorageData();
+import com.fasterxml.jackson.annotation.JsonProperty;
+import java.util.Map;
+import lombok.Data;
+
+@Data
+public final class Document {
+    private boolean found;
 
-    /**
-     * Initialize this object based on the given string data.
-     */
-    void toObject(String data);
+    @JsonProperty("_id")
+    private String id;
 
-    /**
-     * Initialize the object based on the given source.
-     */
-    void copyFrom(T source);
+    @JsonProperty("_source")
+    private Map<String, Object> source;
 }
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/response/Documents.java
similarity index 61%
copy from oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
copy to oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/response/Documents.java
index ebad19e..6084cab 100644
--- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/response/Documents.java
@@ -13,27 +13,20 @@
  * 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.core.storage.type;
+package org.apache.skywalking.library.elasticsearch.response;
 
-/**
- * StorageDataComplexObject implementation supports String-Object interconversion.
- */
-public interface StorageDataComplexObject<T> {
-    /**
-     * @return string representing this object.
-     */
-    String toStorageData();
+import java.util.Iterator;
+import java.util.List;
+import lombok.Data;
 
-    /**
-     * Initialize this object based on the given string data.
-     */
-    void toObject(String data);
+@Data
+public final class Documents implements Iterable<Document> {
+    private List<Document> docs;
 
-    /**
-     * Initialize the object based on the given source.
-     */
-    void copyFrom(T source);
+    @Override
+    public Iterator<Document> iterator() {
+        return docs.stream().filter(Document::isFound).iterator();
+    }
 }
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/response/NodeInfo.java
similarity index 61%
copy from oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
copy to oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/response/NodeInfo.java
index ebad19e..3e1ea2b 100644
--- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/response/NodeInfo.java
@@ -13,27 +13,18 @@
  * 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.core.storage.type;
+package org.apache.skywalking.library.elasticsearch.response;
 
-/**
- * StorageDataComplexObject implementation supports String-Object interconversion.
- */
-public interface StorageDataComplexObject<T> {
-    /**
-     * @return string representing this object.
-     */
-    String toStorageData();
+import lombok.Data;
 
-    /**
-     * Initialize this object based on the given string data.
-     */
-    void toObject(String data);
+@Data
+public final class NodeInfo {
+    @Data
+    public static class Version {
+        private String number;
+    }
 
-    /**
-     * Initialize the object based on the given source.
-     */
-    void copyFrom(T source);
+    private Version version;
 }
diff --git a/oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ElasticSearchInsertRequest.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/response/search/SearchHit.java
similarity index 50%
rename from oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ElasticSearchInsertRequest.java
rename to oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/response/search/SearchHit.java
index 9c0655c..5190e1a 100644
--- a/oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/elasticsearch/ElasticSearchInsertRequest.java
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/response/search/SearchHit.java
@@ -15,21 +15,34 @@
  * limitations under the License.
  */
 
-package org.apache.skywalking.oap.server.library.client.elasticsearch;
+package org.apache.skywalking.library.elasticsearch.response.search;
 
-import org.apache.skywalking.oap.server.library.client.request.InsertRequest;
-import org.elasticsearch.action.index.IndexRequest;
-import org.elasticsearch.common.xcontent.XContentBuilder;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import lombok.Getter;
+import lombok.Setter;
 
-public class ElasticSearchInsertRequest extends IndexRequest implements InsertRequest {
-
-    public ElasticSearchInsertRequest(String index, String type, String id) {
-        super(index, type, id);
-    }
+@Getter
+@Setter
+public final class SearchHit implements Iterable<Map.Entry<String, Object>> {
+    @JsonProperty("_index")
+    private String index;
+    @JsonProperty("_type")
+    private String type;
+    @JsonProperty("_id")
+    private String id;
+    @JsonProperty("_score")
+    private double score;
+    @JsonProperty("_source")
+    private Map<String, Object> source;
 
     @Override
-    public ElasticSearchInsertRequest source(XContentBuilder sourceBuilder) {
-        super.source(sourceBuilder);
-        return this;
+    public Iterator<Map.Entry<String, Object>> iterator() {
+        if (source == null) {
+            return Collections.emptyIterator();
+        }
+        return source.entrySet().iterator();
     }
 }
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/response/search/SearchHits.java
similarity index 61%
copy from oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
copy to oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/response/search/SearchHits.java
index ebad19e..64dc0cf 100644
--- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/response/search/SearchHits.java
@@ -13,27 +13,24 @@
  * 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.core.storage.type;
+package org.apache.skywalking.library.elasticsearch.response.search;
 
-/**
- * StorageDataComplexObject implementation supports String-Object interconversion.
- */
-public interface StorageDataComplexObject<T> {
-    /**
-     * @return string representing this object.
-     */
-    String toStorageData();
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import lombok.Getter;
+import lombok.Setter;
 
-    /**
-     * Initialize this object based on the given string data.
-     */
-    void toObject(String data);
+@Getter
+@Setter
+public final class SearchHits<T> implements Iterable<T> {
+    private int total;
+    private List<T> hits = new ArrayList<>();
 
-    /**
-     * Initialize the object based on the given source.
-     */
-    void copyFrom(T source);
+    @Override
+    public Iterator<T> iterator() {
+        return hits.iterator();
+    }
 }
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/response/search/SearchResponse.java
similarity index 61%
copy from oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
copy to oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/response/search/SearchResponse.java
index ebad19e..587a462 100644
--- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/type/StorageDataComplexObject.java
+++ b/oap-server/server-library/library-elasticsearch-client/src/main/java/org/apache/skywalking/library/elasticsearch/response/search/SearchResponse.java
@@ -13,27 +13,17 @@
  * 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.core.storage.type;
-
-/**
- * StorageDataComplexObject implementation supports String-Object interconversion.
- */
-public interface StorageDataComplexObject<T> {
-    /**
-     * @return string representing this object.
-     */
-    String toStorageData();
+package org.apache.skywalking.library.elasticsearch.response.search;
 
-    /**
-     * Initialize this object based on the given string data.
-     */
-    void toObject(String data);
+import java.util.Map;
+import lombok.Getter;
+import lombok.Setter;
 
-    /**
-     * Initialize the object based on the given source.
-     */
-    void copyFrom(T source);
+@Getter
+@Setter
+public final class SearchResponse {
+    private SearchHits<SearchHit> hits = new SearchHits<>();
+    private Map<String, Object> aggregations;
 }
diff --git a/oap-server/server-library/library-elasticsearch-client/src/test/java/org/apache/skywalking/library/elasticsearch/ITElasticSearchClientTest.java b/oap-server/server-library/library-elasticsearch-client/src/test/java/org/apache/skywalking/library/elasticsearch/ITElasticSearchClientTest.java
new file mode 100644
index 0000000..461e3b6
--- /dev/null
+++ b/oap-server/server-library/library-elasticsearch-client/src/test/java/org/apache/skywalking/library/elasticsearch/ITElasticSearchClientTest.java
@@ -0,0 +1,106 @@
+/*
+ * 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.library.elasticsearch;
+
+import java.util.Arrays;
+import java.util.Collection;
+import lombok.RequiredArgsConstructor;
+import org.apache.skywalking.library.elasticsearch.requests.IndexRequest;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.testcontainers.elasticsearch.ElasticsearchContainer;
+import org.testcontainers.shaded.com.google.common.collect.ImmutableMap;
+import org.testcontainers.utility.DockerImageName;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+@RequiredArgsConstructor
+@RunWith(Parameterized.class)
+public class ITElasticSearchClientTest {
+
+    @Parameterized.Parameters(name = "version: {0}")
+    public static Collection<Object[]> versions() {
+        return Arrays.asList(new Object[][] {
+            {"6.3.2"} // TODO: add 7.x version
+        });
+    }
+
+    private final String version;
+
+    private ElasticsearchContainer server;
+    private ElasticSearchClient client;
+
+    @Before
+    public void setup() {
+        server = new ElasticsearchContainer(
+            DockerImageName.parse("docker.elastic.co/elasticsearch/elasticsearch-oss")
+                           .withTag(version)
+        );
+        server.start();
+
+        client = ElasticSearchClient.builder()
+                                    .endpoints(server.getHttpHostAddress())
+                                    .build();
+        client.connect();
+    }
+
+    @After
+    public void tearDown() {
+        server.stop();
+    }
+
+    @Test
+    public void testIndex() {
+        final String index = "test-index";
+        assertFalse(client.index().exists(index));
+        assertTrue(client.index().get(index).isEmpty());
+        assertTrue(client.index().create(index));
+        assertTrue(client.index().exists(index));
+        assertNotNull(client.index().get(index));
+        assertTrue(client.index().delete(index));
+        assertTrue(client.index().get(index).isEmpty());
+    }
+
+    @Test
+    public void testDoc() {
+        final String index = "test-index";
+        assertTrue(client.index().create(index));
+
+        final ImmutableMap<String, Object> doc = ImmutableMap.of("key", "val");
+        final String idWithSpace = "an id"; // UI management templates' IDs contains spaces
+        final String type = "type";
+
+        client.documents().index(
+            IndexRequest.builder()
+                        .index(index)
+                        .type(type)
+                        .id(idWithSpace)
+                        .doc(doc)
+                        .build(), null);
+
+        assertTrue(client.documents().get(index, type, idWithSpace).isPresent());
+        assertEquals(client.documents().get(index, type, idWithSpace).get().getId(), idWithSpace);
+        assertEquals(client.documents().get(index, type, idWithSpace).get().getSource(), doc);
+    }
+}
diff --git a/oap-server/server-library/pom.xml b/oap-server/server-library/pom.xml
index acfcd44..fa37d28 100644
--- a/oap-server/server-library/pom.xml
+++ b/oap-server/server-library/pom.xml
@@ -32,6 +32,7 @@
         <module>library-server</module>
         <module>library-util</module>
         <module>library-client</module>
+        <module>library-elasticsearch-client</module>
     </modules>
 
     <dependencies>
@@ -40,4 +41,4 @@
             <artifactId>apm-util</artifactId>
         </dependency>
     </dependencies>
-</project>
\ No newline at end of file
+</project>
diff --git a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/StorageModuleElasticsearchProvider.java b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/StorageModuleElasticsearchProvider.java
index ce76619..a9088a0 100644
--- a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/StorageModuleElasticsearchProvider.java
+++ b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/StorageModuleElasticsearchProvider.java
@@ -229,6 +229,7 @@ public class StorageModuleElasticsearchProvider extends ModuleProvider {
         return new String[] {CoreModule.NAME};
     }
 
+    // TODO
     public static List<IndexNameConverter> indexNameConverters(String namespace) {
         List<IndexNameConverter> converters = new ArrayList<>();
         converters.add(new NamespaceConverter(namespace));
diff --git a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/BatchProcessEsDAO.java b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/BatchProcessEsDAO.java
index fb8df02..0cc371d 100644
--- a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/BatchProcessEsDAO.java
+++ b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/BatchProcessEsDAO.java
@@ -20,14 +20,14 @@ package org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base;
 
 import java.util.List;
 import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.library.elasticsearch.bulk.BulkProcessor;
 import org.apache.skywalking.oap.server.core.storage.IBatchDAO;
 import org.apache.skywalking.oap.server.library.client.elasticsearch.ElasticSearchClient;
+import org.apache.skywalking.oap.server.library.client.elasticsearch.IndexRequestWrapper;
+import org.apache.skywalking.oap.server.library.client.elasticsearch.UpdateRequestWrapper;
 import org.apache.skywalking.oap.server.library.client.request.InsertRequest;
 import org.apache.skywalking.oap.server.library.client.request.PrepareRequest;
 import org.apache.skywalking.oap.server.library.util.CollectionUtils;
-import org.elasticsearch.action.bulk.BulkProcessor;
-import org.elasticsearch.action.index.IndexRequest;
-import org.elasticsearch.action.update.UpdateRequest;
 
 @Slf4j
 public class BatchProcessEsDAO extends EsDAO implements IBatchDAO {
@@ -52,7 +52,7 @@ public class BatchProcessEsDAO extends EsDAO implements IBatchDAO {
             this.bulkProcessor = getClient().createBulkProcessor(bulkActions, flushInterval, concurrentRequests);
         }
 
-        this.bulkProcessor.add((IndexRequest) insertRequest);
+        this.bulkProcessor.add(((IndexRequestWrapper) insertRequest).getRequest());
     }
 
     @Override
@@ -64,9 +64,9 @@ public class BatchProcessEsDAO extends EsDAO implements IBatchDAO {
         if (CollectionUtils.isNotEmpty(prepareRequests)) {
             for (PrepareRequest prepareRequest : prepareRequests) {
                 if (prepareRequest instanceof InsertRequest) {
-                    this.bulkProcessor.add((IndexRequest) prepareRequest);
+                    this.bulkProcessor.add(((IndexRequestWrapper) prepareRequest).getRequest());
                 } else {
-                    this.bulkProcessor.add((UpdateRequest) prepareRequest);
+                    this.bulkProcessor.add(((UpdateRequestWrapper) prepareRequest).getRequest());
                 }
             }
         }
diff --git a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/EsDAO.java b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/EsDAO.java
index f586b6f..e798720 100644
--- a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/EsDAO.java
+++ b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/EsDAO.java
@@ -33,6 +33,7 @@ public abstract class EsDAO extends AbstractDAO<ElasticSearchClient> {
         super(client);
     }
 
+    // TODO: remove
     protected XContentBuilder map2builder(Map<String, Object> objectMap) throws IOException {
         XContentBuilder builder = XContentFactory.jsonBuilder().startObject();
         for (Map.Entry<String, Object> entries: objectMap.entrySet()) {
diff --git a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/HistoryDeleteEsDAO.java b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/HistoryDeleteEsDAO.java
index af73fc9..5f687a1 100644
--- a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/HistoryDeleteEsDAO.java
+++ b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/HistoryDeleteEsDAO.java
@@ -20,6 +20,7 @@ package org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.skywalking.oap.server.core.analysis.DownSampling;
@@ -53,7 +54,7 @@ public class HistoryDeleteEsDAO extends EsDAO implements IHistoryDeleteDAO {
         }
         deadline = Long.parseLong(new DateTime().plusDays(-ttl).toString("yyyyMMdd"));
         String tableName = IndexController.INSTANCE.getTableName(model);
-        List<String> indexes = client.retrievalIndexByAliases(tableName);
+        Collection<String> indexes = client.retrievalIndexByAliases(tableName);
 
         List<String> prepareDeleteIndexes = new ArrayList<>();
         List<String> leftIndices = new ArrayList<>();
@@ -95,7 +96,7 @@ public class HistoryDeleteEsDAO extends EsDAO implements IHistoryDeleteDAO {
                 }
             }
             String tableName = IndexController.INSTANCE.getTableName(model);
-            List<String> indexes;
+            Collection<String> indexes;
             try {
                 indexes = client.retrievalIndexByAliases(tableName);
             } catch (IOException e) {
diff --git a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/ManagementEsDAO.java b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/ManagementEsDAO.java
index 2c8676c..2925d0f 100644
--- a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/ManagementEsDAO.java
+++ b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/ManagementEsDAO.java
@@ -19,13 +19,12 @@
 package org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base;
 
 import java.io.IOException;
+import java.util.Map;
 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.StorageHashMapBuilder;
 import org.apache.skywalking.oap.server.core.storage.model.Model;
 import org.apache.skywalking.oap.server.library.client.elasticsearch.ElasticSearchClient;
-import org.elasticsearch.action.get.GetResponse;
-import org.elasticsearch.common.xcontent.XContentBuilder;
 
 public class ManagementEsDAO extends EsDAO implements IManagementDAO {
     private final StorageHashMapBuilder<ManagementData> storageBuilder;
@@ -40,12 +39,13 @@ public class ManagementEsDAO extends EsDAO implements IManagementDAO {
     public void insert(Model model, ManagementData managementData) throws IOException {
         String tableName = IndexController.INSTANCE.getTableName(model);
         String docId = IndexController.INSTANCE.generateDocId(model, managementData.id());
-        final GetResponse response = getClient().get(tableName, docId);
-        if (response.isExists()) {
+        final boolean exist = getClient().existDoc(tableName, docId);
+        if (exist) {
             return;
         }
-        XContentBuilder builder = map2builder(
-            IndexController.INSTANCE.appendMetricTableColumn(model, storageBuilder.entity2Storage(managementData)));
-        getClient().forceInsert(tableName, docId, builder);
+        Map<String, Object> source =
+            IndexController.INSTANCE.appendMetricTableColumn(model, storageBuilder.entity2Storage(
+                managementData));
+        getClient().forceInsert(tableName, docId, source);
     }
-}
\ No newline at end of file
+}
diff --git a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/MetricsEsDAO.java b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/MetricsEsDAO.java
index 38d379d..ce96138 100644
--- a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/MetricsEsDAO.java
+++ b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/MetricsEsDAO.java
@@ -20,11 +20,11 @@ package org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.library.elasticsearch.response.Document;
 import org.apache.skywalking.oap.server.core.analysis.DownSampling;
 import org.apache.skywalking.oap.server.core.analysis.TimeBucket;
 import org.apache.skywalking.oap.server.core.analysis.metrics.Metrics;
@@ -35,8 +35,6 @@ import org.apache.skywalking.oap.server.library.client.elasticsearch.ElasticSear
 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.elasticsearch.IndicesMetadataCache;
-import org.elasticsearch.action.search.SearchResponse;
-import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.joda.time.DateTime;
 
 import static java.util.stream.Collectors.groupingBy;
@@ -90,15 +88,12 @@ public class MetricsEsDAO extends EsDAO implements IMetricsDAO {
             String[] ids = metricList.stream()
                                      .map(item -> IndexController.INSTANCE.generateDocId(model, item.id()))
                                      .toArray(String[]::new);
-            try {
-                SearchResponse response = getClient().ids(tableName, ids);
-                for (int i = 0; i < response.getHits().getHits().length; i++) {
-                    Metrics source = storageBuilder.storage2Entity(response.getHits().getAt(i).getSourceAsMap());
+            getClient().ids(tableName, ids).ifPresent(documents -> {
+                for (final Document doc : documents) {
+                    Metrics source = storageBuilder.storage2Entity(doc.getSource());
                     result.add(source);
                 }
-            } catch (IOException e) {
-                log.error("multiGet id=" + Arrays.toString(ids) + " from " + tableName + " fails.", e);
-            }
+            });
         });
 
         return result;
@@ -106,8 +101,8 @@ public class MetricsEsDAO extends EsDAO implements IMetricsDAO {
 
     @Override
     public InsertRequest prepareBatchInsert(Model model, Metrics metrics) throws IOException {
-        XContentBuilder builder = map2builder(
-            IndexController.INSTANCE.appendMetricTableColumn(model, storageBuilder.entity2Storage(metrics)));
+        Map<String, Object> builder =
+            IndexController.INSTANCE.appendMetricTableColumn(model, storageBuilder.entity2Storage(metrics));
         String modelName = TimeSeriesUtils.writeIndexName(model, metrics.getTimeBucket());
         String id = IndexController.INSTANCE.generateDocId(model, metrics.id());
         return getClient().prepareInsert(modelName, id, builder);
@@ -115,8 +110,8 @@ public class MetricsEsDAO extends EsDAO implements IMetricsDAO {
 
     @Override
     public UpdateRequest prepareBatchUpdate(Model model, Metrics metrics) throws IOException {
-        XContentBuilder builder = map2builder(
-            IndexController.INSTANCE.appendMetricTableColumn(model, storageBuilder.entity2Storage(metrics)));
+        Map<String, Object> builder =
+            IndexController.INSTANCE.appendMetricTableColumn(model, storageBuilder.entity2Storage(metrics));
         String modelName = TimeSeriesUtils.writeIndexName(model, metrics.getTimeBucket());
         String id = IndexController.INSTANCE.generateDocId(model, metrics.id());
         return getClient().prepareUpdate(modelName, id, builder);
diff --git a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/NoneStreamEsDAO.java b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/NoneStreamEsDAO.java
index 3d89243..bb92956 100644
--- a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/NoneStreamEsDAO.java
+++ b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/NoneStreamEsDAO.java
@@ -19,12 +19,12 @@
 package org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base;
 
 import java.io.IOException;
+import java.util.Map;
 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.StorageHashMapBuilder;
 import org.apache.skywalking.oap.server.core.storage.model.Model;
 import org.apache.skywalking.oap.server.library.client.elasticsearch.ElasticSearchClient;
-import org.elasticsearch.common.xcontent.XContentBuilder;
 
 /**
  * Synchronize storage Elasticsearch implements
@@ -40,8 +40,9 @@ public class NoneStreamEsDAO extends EsDAO implements INoneStreamDAO {
 
     @Override
     public void insert(Model model, NoneStream noneStream) throws IOException {
-        XContentBuilder builder = map2builder(
-            IndexController.INSTANCE.appendMetricTableColumn(model, storageBuilder.entity2Storage(noneStream)));
+        Map<String, Object> builder =
+            IndexController.INSTANCE.appendMetricTableColumn(model, storageBuilder.entity2Storage(
+                noneStream));
         String modelName = TimeSeriesUtils.writeIndexName(model, noneStream.getTimeBucket());
         String id = IndexController.INSTANCE.generateDocId(model, noneStream.id());
         getClient().forceInsert(modelName, id, builder);
diff --git a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/RecordEsDAO.java b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/RecordEsDAO.java
index f23bc61..26f35c7 100644
--- a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/RecordEsDAO.java
+++ b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/RecordEsDAO.java
@@ -19,13 +19,13 @@
 package org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base;
 
 import java.io.IOException;
+import java.util.Map;
 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.elasticsearch.ElasticSearchClient;
 import org.apache.skywalking.oap.server.library.client.request.InsertRequest;
-import org.elasticsearch.common.xcontent.XContentBuilder;
 
 public class RecordEsDAO extends EsDAO implements IRecordDAO {
     private final StorageHashMapBuilder<Record> storageBuilder;
@@ -38,8 +38,8 @@ public class RecordEsDAO extends EsDAO implements IRecordDAO {
 
     @Override
     public InsertRequest prepareBatchInsert(Model model, Record record) throws IOException {
-        XContentBuilder builder = map2builder(
-            IndexController.INSTANCE.appendMetricTableColumn(model, storageBuilder.entity2Storage(record)));
+        Map<String, Object> builder =
+            IndexController.INSTANCE.appendMetricTableColumn(model, storageBuilder.entity2Storage(record));
         String modelName = TimeSeriesUtils.writeIndexName(model, record.getTimeBucket());
         String id = IndexController.INSTANCE.generateDocId(model, record.id());
         return getClient().prepareInsert(modelName, id, builder);
diff --git a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/cache/NetworkAddressAliasEsDAO.java b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/cache/NetworkAddressAliasEsDAO.java
index 408839c..d58f6ce 100644
--- a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/cache/NetworkAddressAliasEsDAO.java
+++ b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/cache/NetworkAddressAliasEsDAO.java
@@ -21,14 +21,13 @@ package org.apache.skywalking.oap.server.storage.plugin.elasticsearch.cache;
 import java.util.ArrayList;
 import java.util.List;
 import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.library.elasticsearch.requests.search.Query;
+import org.apache.skywalking.library.elasticsearch.requests.search.Search;
+import org.apache.skywalking.library.elasticsearch.response.search.SearchHit;
 import org.apache.skywalking.oap.server.core.analysis.manual.networkalias.NetworkAddressAlias;
 import org.apache.skywalking.oap.server.core.storage.cache.INetworkAddressAliasDAO;
 import org.apache.skywalking.oap.server.library.client.elasticsearch.ElasticSearchClient;
 import org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base.EsDAO;
-import org.elasticsearch.action.search.SearchResponse;
-import org.elasticsearch.index.query.QueryBuilders;
-import org.elasticsearch.search.SearchHit;
-import org.elasticsearch.search.builder.SearchSourceBuilder;
 
 @Slf4j
 public class NetworkAddressAliasEsDAO extends EsDAO implements INetworkAddressAliasDAO {
@@ -44,16 +43,19 @@ public class NetworkAddressAliasEsDAO extends EsDAO implements INetworkAddressAl
         List<NetworkAddressAlias> networkAddressAliases = new ArrayList<>();
 
         try {
-            SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder();
-            searchSourceBuilder.query(QueryBuilders.rangeQuery(NetworkAddressAlias.LAST_UPDATE_TIME_BUCKET)
-                                                   .gte(timeBucketInMinute));
-            searchSourceBuilder.size(resultWindowMaxSize);
+            final Search search =
+                Search.builder().query(
+                          Query.range(NetworkAddressAlias.LAST_UPDATE_TIME_BUCKET)
+                               .gte(timeBucketInMinute))
+                      .size(resultWindowMaxSize)
+                      .build();
 
-            SearchResponse response = getClient().search(NetworkAddressAlias.INDEX_NAME, searchSourceBuilder);
+            final org.apache.skywalking.library.elasticsearch.response.search.SearchResponse results =
+                getClient().search(NetworkAddressAlias.INDEX_NAME, search);
 
             final NetworkAddressAlias.Builder builder = new NetworkAddressAlias.Builder();
-            for (SearchHit searchHit : response.getHits().getHits()) {
-                networkAddressAliases.add(builder.storage2Entity(searchHit.getSourceAsMap()));
+            for (SearchHit searchHit : results.getHits().getHits()) {
+                networkAddressAliases.add(builder.storage2Entity(searchHit.getSource()));
             }
         } catch (Throwable t) {
             log.error(t.getMessage(), t);
diff --git a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/AggregationQueryEsDAO.java b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/AggregationQueryEsDAO.java
index 65efcbd..3850513 100644
--- a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/AggregationQueryEsDAO.java
+++ b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/AggregationQueryEsDAO.java
@@ -21,6 +21,15 @@ package org.apache.skywalking.oap.server.storage.plugin.elasticsearch.query;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
+import org.apache.skywalking.library.elasticsearch.requests.search.BoolQueryBuilder;
+import org.apache.skywalking.library.elasticsearch.requests.search.Query;
+import org.apache.skywalking.library.elasticsearch.requests.search.RangeQueryBuilder;
+import org.apache.skywalking.library.elasticsearch.requests.search.Search;
+import org.apache.skywalking.library.elasticsearch.requests.search.SearchBuilder;
+import org.apache.skywalking.library.elasticsearch.requests.search.aggregation.Aggregation;
+import org.apache.skywalking.library.elasticsearch.requests.search.aggregation.BucketOrder;
+import org.apache.skywalking.library.elasticsearch.response.search.SearchResponse;
 import org.apache.skywalking.oap.server.core.analysis.metrics.Metrics;
 import org.apache.skywalking.oap.server.core.query.enumeration.Order;
 import org.apache.skywalking.oap.server.core.query.input.Duration;
@@ -32,15 +41,6 @@ import org.apache.skywalking.oap.server.library.client.elasticsearch.ElasticSear
 import org.apache.skywalking.oap.server.library.util.CollectionUtils;
 import org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base.EsDAO;
 import org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base.IndexController;
-import org.elasticsearch.action.search.SearchResponse;
-import org.elasticsearch.index.query.BoolQueryBuilder;
-import org.elasticsearch.index.query.QueryBuilders;
-import org.elasticsearch.index.query.RangeQueryBuilder;
-import org.elasticsearch.search.aggregations.AggregationBuilders;
-import org.elasticsearch.search.aggregations.BucketOrder;
-import org.elasticsearch.search.aggregations.bucket.terms.Terms;
-import org.elasticsearch.search.aggregations.metrics.avg.Avg;
-import org.elasticsearch.search.builder.SearchSourceBuilder;
 
 public class AggregationQueryEsDAO extends EsDAO implements IAggregationQueryDAO {
 
@@ -52,73 +52,78 @@ public class AggregationQueryEsDAO extends EsDAO implements IAggregationQueryDAO
     public List<SelectedRecord> sortMetrics(final TopNCondition condition,
                                             final String valueColumnName,
                                             final Duration duration,
-                                            final List<KeyValue> additionalConditions) throws IOException {
-        SearchSourceBuilder sourceBuilder = SearchSourceBuilder.searchSource();
-        final RangeQueryBuilder queryBuilder = QueryBuilders.rangeQuery(Metrics.TIME_BUCKET)
-                                                            .lte(duration.getEndTimeBucket())
-                                                            .gte(duration.getStartTimeBucket());
+                                            final List<KeyValue> additionalConditions)
+        throws IOException {
+        final RangeQueryBuilder basicQuery = Query.range(Metrics.TIME_BUCKET)
+                                                  .lte(duration.getEndTimeBucket())
+                                                  .gte(duration.getStartTimeBucket());
+        final SearchBuilder search = Search.builder();
 
-        boolean asc = false;
-        if (condition.getOrder().equals(Order.ASC)) {
-            asc = true;
-        }
-        String tableName = IndexController.LogicIndicesRegister.getPhysicalTableName(condition.getName());
+        final boolean asc = condition.getOrder().equals(Order.ASC);
+        final String tableName =
+            IndexController.LogicIndicesRegister.getPhysicalTableName(condition.getName());
 
         if (CollectionUtils.isEmpty(additionalConditions)
             && IndexController.LogicIndicesRegister.isMetricTable(condition.getName())) {
-            BoolQueryBuilder boolQuery = QueryBuilders.boolQuery();
-            boolQuery.must()
-                     .add(QueryBuilders.termQuery(
+            final org.apache.skywalking.library.elasticsearch.requests.search.BoolQueryBuilder
+                boolQuery = Query.bool();
+            boolQuery.must(basicQuery)
+                     .must(Query.term(
                          IndexController.LogicIndicesRegister.METRIC_TABLE_NAME,
                          condition.getName()
                      ));
-            boolQuery.must().add(queryBuilder);
-            sourceBuilder.query(boolQuery);
+            search.query(boolQuery);
         } else if (CollectionUtils.isEmpty(additionalConditions)) {
-            sourceBuilder.query(queryBuilder);
+            search.query(basicQuery);
         } else if (CollectionUtils.isNotEmpty(additionalConditions)
             && IndexController.LogicIndicesRegister.isMetricTable(condition.getName())) {
-            BoolQueryBuilder boolQuery = QueryBuilders.boolQuery();
-            boolQuery.must()
-                     .add(QueryBuilders.termQuery(
-                         IndexController.LogicIndicesRegister.METRIC_TABLE_NAME,
-                         condition.getName()
-                     ));
+            final org.apache.skywalking.library.elasticsearch.requests.search.BoolQueryBuilder
+                boolQuery = Query.bool();
+            boolQuery.must(Query.term(
+                IndexController.LogicIndicesRegister.METRIC_TABLE_NAME,
+                condition.getName()
+            ));
             additionalConditions.forEach(additionalCondition -> boolQuery
-                .must()
-                .add(QueryBuilders.termsQuery(additionalCondition.getKey(), additionalCondition.getValue())));
-            boolQuery.must().add(queryBuilder);
-            sourceBuilder.query(boolQuery);
+                .must(Query.term(
+                    additionalCondition.getKey(),
+                    additionalCondition.getValue()
+                )));
+            boolQuery.must(basicQuery);
+            search.query(boolQuery);
         } else {
-            BoolQueryBuilder boolQuery = QueryBuilders.boolQuery();
+            final BoolQueryBuilder boolQuery = Query.bool();
             additionalConditions.forEach(additionalCondition -> boolQuery
-                .must()
-                .add(QueryBuilders.termsQuery(additionalCondition.getKey(), additionalCondition.getValue())));
-            boolQuery.must().add(queryBuilder);
-            sourceBuilder.query(boolQuery);
+                .must(Query.term(
+                    additionalCondition.getKey(),
+                    additionalCondition.getValue()
+                )));
+            boolQuery.must(basicQuery);
+            search.query(boolQuery);
         }
 
-        sourceBuilder.aggregation(
-            AggregationBuilders.terms(Metrics.ENTITY_ID)
-                               .field(Metrics.ENTITY_ID)
-                               .order(BucketOrder.aggregation(valueColumnName, asc))
-                               .size(condition.getTopN())
-                               .subAggregation(AggregationBuilders.avg(valueColumnName).field(valueColumnName))
-        );
+        search.aggregation(
+            Aggregation.terms(Metrics.ENTITY_ID)
+                       .field(Metrics.ENTITY_ID)
+                       .order(BucketOrder.aggregation(valueColumnName, asc))
+                       .size(condition.getTopN())
+                       .subAggregation(Aggregation.avg(valueColumnName).field(valueColumnName))
+                       .build());
 
-        SearchResponse response = getClient().search(tableName, sourceBuilder);
+        final SearchResponse response = getClient().search(tableName, search.build());
 
-        List<SelectedRecord> topNList = new ArrayList<>();
-        Terms idTerms = response.getAggregations().get(Metrics.ENTITY_ID);
-        for (Terms.Bucket termsBucket : idTerms.getBuckets()) {
+        final List<SelectedRecord> topNList = new ArrayList<>();
+        final Map<String, Object> idTerms =
+            (Map<String, Object>) response.getAggregations().get(Metrics.ENTITY_ID);
+        final List<Map<String, Object>> buckets =
+            (List<Map<String, Object>>) idTerms.get("buckets");
+        for (Map<String, Object> termsBucket : buckets) {
             SelectedRecord record = new SelectedRecord();
-            record.setId(termsBucket.getKeyAsString());
-            Avg value = termsBucket.getAggregations().get(valueColumnName);
-            record.setValue(String.valueOf((long) value.getValue()));
+            record.setId((String) termsBucket.get("key"));
+            Map<String, Object> value = (Map<String, Object>) termsBucket.get(valueColumnName);
+            record.setValue(String.valueOf(value.get("value")));
             topNList.add(record);
         }
 
         return topNList;
     }
-
 }
diff --git a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/AlarmQueryEsDAO.java b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/AlarmQueryEsDAO.java
index d1a0cf4..c1bbb63 100644
--- a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/AlarmQueryEsDAO.java
+++ b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/AlarmQueryEsDAO.java
@@ -19,11 +19,15 @@
 package org.apache.skywalking.oap.server.storage.plugin.elasticsearch.query;
 
 import com.google.common.base.Strings;
-
 import java.io.IOException;
 import java.util.List;
 import java.util.Objects;
-
+import org.apache.skywalking.library.elasticsearch.requests.search.Query;
+import org.apache.skywalking.library.elasticsearch.requests.search.Search;
+import org.apache.skywalking.library.elasticsearch.requests.search.SearchBuilder;
+import org.apache.skywalking.library.elasticsearch.requests.search.Sort;
+import org.apache.skywalking.library.elasticsearch.response.search.SearchHit;
+import org.apache.skywalking.library.elasticsearch.response.search.SearchResponse;
 import org.apache.skywalking.oap.server.core.alarm.AlarmRecord;
 import org.apache.skywalking.oap.server.core.analysis.manual.searchtag.Tag;
 import org.apache.skywalking.oap.server.core.query.enumeration.Scope;
@@ -35,13 +39,6 @@ import org.apache.skywalking.oap.server.library.util.CollectionUtils;
 import org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base.EsDAO;
 import org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base.IndexController;
 import org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base.MatchCNameBuilder;
-import org.elasticsearch.action.search.SearchResponse;
-import org.elasticsearch.index.query.BoolQueryBuilder;
-import org.elasticsearch.index.query.QueryBuilder;
-import org.elasticsearch.index.query.QueryBuilders;
-import org.elasticsearch.search.SearchHit;
-import org.elasticsearch.search.builder.SearchSourceBuilder;
-import org.elasticsearch.search.sort.SortOrder;
 
 public class AlarmQueryEsDAO extends EsDAO implements IAlarmQueryDAO {
 
@@ -50,43 +47,46 @@ public class AlarmQueryEsDAO extends EsDAO implements IAlarmQueryDAO {
     }
 
     @Override
-    public Alarms getAlarm(final Integer scopeId, final String keyword, final int limit, final int from,
-                           final long startTB, final long endTB, final List<Tag> tags) throws IOException {
-        SearchSourceBuilder sourceBuilder = SearchSourceBuilder.searchSource();
+    public Alarms getAlarm(final Integer scopeId, final String keyword, final int limit,
+                           final int from,
+                           final long startTB, final long endTB, final List<Tag> tags)
+        throws IOException {
+        final String index =
+            IndexController.LogicIndicesRegister.getPhysicalTableName(AlarmRecord.INDEX_NAME);
 
-        BoolQueryBuilder boolQueryBuilder = QueryBuilders.boolQuery();
-        List<QueryBuilder> mustQueryList = boolQueryBuilder.must();
+        final org.apache.skywalking.library.elasticsearch.requests.search.BoolQueryBuilder query =
+            Query.bool();
 
         if (startTB != 0 && endTB != 0) {
-            mustQueryList.add(QueryBuilders.rangeQuery(AlarmRecord.TIME_BUCKET).gte(startTB).lte(endTB));
+            query.must(Query.range(AlarmRecord.TIME_BUCKET).gte(startTB).lte(endTB));
         }
 
         if (Objects.nonNull(scopeId)) {
-            mustQueryList.add(QueryBuilders.termQuery(AlarmRecord.SCOPE, scopeId.intValue()));
+            query.must(Query.term(AlarmRecord.SCOPE, scopeId));
         }
 
         if (!Strings.isNullOrEmpty(keyword)) {
             String matchCName = MatchCNameBuilder.INSTANCE.build(AlarmRecord.ALARM_MESSAGE);
-            mustQueryList.add(QueryBuilders.matchPhraseQuery(matchCName, keyword));
+            query.must(Query.matchPhrase(matchCName, keyword));
         }
 
         if (CollectionUtils.isNotEmpty(tags)) {
-            tags.forEach(tag -> mustQueryList.add(QueryBuilders.termQuery(AlarmRecord.TAGS, tag.toString())));
+            tags.forEach(tag -> query.must(Query.term(AlarmRecord.TAGS, tag.toString())));
         }
 
-        sourceBuilder.query(boolQueryBuilder).sort(AlarmRecord.START_TIME, SortOrder.DESC);
-        sourceBuilder.size(limit);
-        sourceBuilder.from(from);
+        final SearchBuilder search =
+            Search.builder().query(query)
+                  .size(limit).from(from)
+                  .sort(AlarmRecord.START_TIME, Sort.Order.DESC);
 
-        SearchResponse response = getClient()
-                .search(IndexController.LogicIndicesRegister.getPhysicalTableName(AlarmRecord.INDEX_NAME), sourceBuilder);
+        SearchResponse response = getClient().search(index, search.build());
 
         Alarms alarms = new Alarms();
-        alarms.setTotal((int) response.getHits().totalHits);
+        alarms.setTotal(response.getHits().getTotal());
 
         for (SearchHit searchHit : response.getHits().getHits()) {
             AlarmRecord.Builder builder = new AlarmRecord.Builder();
-            AlarmRecord alarmRecord = builder.storage2Entity(searchHit.getSourceAsMap());
+            AlarmRecord alarmRecord = builder.storage2Entity(searchHit.getSource());
 
             AlarmMessage message = new AlarmMessage();
             message.setId(String.valueOf(alarmRecord.getId0()));
diff --git a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/ESEventQueryDAO.java b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/ESEventQueryDAO.java
index fb7c865..0701815 100644
--- a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/ESEventQueryDAO.java
+++ b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/ESEventQueryDAO.java
@@ -22,27 +22,26 @@ import java.io.IOException;
 import java.util.List;
 import java.util.Objects;
 import java.util.stream.Collectors;
-import java.util.stream.Stream;
+import org.apache.skywalking.library.elasticsearch.requests.search.BoolQueryBuilder;
+import org.apache.skywalking.library.elasticsearch.requests.search.Query;
+import org.apache.skywalking.library.elasticsearch.requests.search.Search;
+import org.apache.skywalking.library.elasticsearch.requests.search.SearchBuilder;
+import org.apache.skywalking.library.elasticsearch.requests.search.Sort;
+import org.apache.skywalking.library.elasticsearch.response.search.SearchHit;
+import org.apache.skywalking.library.elasticsearch.response.search.SearchResponse;
 import org.apache.skywalking.oap.server.core.query.PaginationUtils;
-import org.apache.skywalking.oap.server.core.source.Event;
 import org.apache.skywalking.oap.server.core.query.enumeration.Order;
 import org.apache.skywalking.oap.server.core.query.input.Duration;
 import org.apache.skywalking.oap.server.core.query.type.event.EventQueryCondition;
 import org.apache.skywalking.oap.server.core.query.type.event.EventType;
 import org.apache.skywalking.oap.server.core.query.type.event.Events;
 import org.apache.skywalking.oap.server.core.query.type.event.Source;
+import org.apache.skywalking.oap.server.core.source.Event;
 import org.apache.skywalking.oap.server.core.storage.query.IEventQueryDAO;
 import org.apache.skywalking.oap.server.library.client.elasticsearch.ElasticSearchClient;
 import org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base.EsDAO;
 import org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base.IndexController;
 import org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base.MatchCNameBuilder;
-import org.elasticsearch.action.search.SearchResponse;
-import org.elasticsearch.index.query.BoolQueryBuilder;
-import org.elasticsearch.index.query.QueryBuilder;
-import org.elasticsearch.index.query.QueryBuilders;
-import org.elasticsearch.search.SearchHit;
-import org.elasticsearch.search.builder.SearchSourceBuilder;
-import org.elasticsearch.search.sort.SortOrder;
 
 import static com.google.common.base.Strings.isNullOrEmpty;
 import static java.util.Objects.isNull;
@@ -54,124 +53,128 @@ public class ESEventQueryDAO extends EsDAO implements IEventQueryDAO {
 
     @Override
     public Events queryEvents(final EventQueryCondition condition) throws Exception {
-        final SearchSourceBuilder sourceBuilder = buildQuery(condition);
+        final SearchBuilder sourceBuilder = buildQuery(condition);
         return getEventsResultByCurrentBuilder(sourceBuilder);
     }
 
     @Override
     public Events queryEvents(List<EventQueryCondition> conditionList) throws Exception {
-        final SearchSourceBuilder sourceBuilder = buildQuery(conditionList);
+        final SearchBuilder sourceBuilder = buildQuery(conditionList);
         return getEventsResultByCurrentBuilder(sourceBuilder);
     }
 
-    private Events getEventsResultByCurrentBuilder(final SearchSourceBuilder sourceBuilder) throws IOException {
-        final SearchResponse response = getClient()
-                .search(IndexController.LogicIndicesRegister.getPhysicalTableName(Event.INDEX_NAME), sourceBuilder);
+    private Events getEventsResultByCurrentBuilder(final SearchBuilder searchBuilder)
+        throws IOException {
+        final String index =
+            IndexController.LogicIndicesRegister.getPhysicalTableName(Event.INDEX_NAME);
+        final SearchResponse response = getClient().search(index, searchBuilder.build());
         final Events events = new Events();
-        events.setTotal((int) response.getHits().totalHits);
-        events.setEvents(Stream.of(response.getHits().getHits())
-                .map(this::parseSearchHit)
-                .collect(Collectors.toList()));
+        events.setTotal(response.getHits().getTotal());
+        events.setEvents(response.getHits().getHits().stream()
+                                 .map(this::parseSearchHit)
+                                 .collect(Collectors.toList()));
         return events;
     }
 
-    private void buildMustQueryListByCondition(final EventQueryCondition condition, final List<QueryBuilder> mustQueryList) {
+    private void buildMustQueryListByCondition(final EventQueryCondition condition,
+                                               final BoolQueryBuilder query) {
         if (!isNullOrEmpty(condition.getUuid())) {
-            mustQueryList.add(QueryBuilders.termQuery(Event.UUID, condition.getUuid()));
+            query.must(Query.term(Event.UUID, condition.getUuid()));
         }
 
         final Source source = condition.getSource();
         if (source != null) {
             if (!isNullOrEmpty(source.getService())) {
-                mustQueryList.add(QueryBuilders.termQuery(Event.SERVICE, source.getService()));
+                query.must(Query.term(Event.SERVICE, source.getService()));
             }
             if (!isNullOrEmpty(source.getServiceInstance())) {
-                mustQueryList.add(QueryBuilders.termQuery(Event.SERVICE_INSTANCE, source.getServiceInstance()));
+                query.must(Query.term(Event.SERVICE_INSTANCE, source.getServiceInstance()));
             }
             if (!isNullOrEmpty(source.getEndpoint())) {
-                mustQueryList.add(QueryBuilders.matchPhraseQuery(
-                        MatchCNameBuilder.INSTANCE.build(Event.ENDPOINT),
-                        source.getEndpoint()
+                query.must(Query.matchPhrase(
+                    MatchCNameBuilder.INSTANCE.build(Event.ENDPOINT),
+                    source.getEndpoint()
                 ));
             }
         }
 
         if (!isNullOrEmpty(condition.getName())) {
-            mustQueryList.add(QueryBuilders.termQuery(Event.NAME, condition.getName()));
+            query.must(Query.term(Event.NAME, condition.getName()));
         }
 
         if (condition.getType() != null) {
-            mustQueryList.add(QueryBuilders.termQuery(Event.TYPE, condition.getType().name()));
+            query.must(Query.term(Event.TYPE, condition.getType().name()));
         }
 
         final Duration startTime = condition.getTime();
         if (startTime != null) {
             if (startTime.getStartTimestamp() > 0) {
-                mustQueryList.add(QueryBuilders.rangeQuery(Event.START_TIME)
-                        .gt(startTime.getStartTimestamp()));
+                query.must(Query.range(Event.START_TIME).gt(startTime.getStartTimestamp()));
             }
             if (startTime.getEndTimestamp() > 0) {
-                mustQueryList.add(QueryBuilders.rangeQuery(Event.END_TIME)
-                        .lt(startTime.getEndTimestamp()));
+                query.must(Query.range(Event.END_TIME).lt(startTime.getEndTimestamp()));
             }
         }
     }
 
-    protected SearchSourceBuilder buildQuery(final List<EventQueryCondition> conditionList) {
-        final SearchSourceBuilder sourceBuilder = SearchSourceBuilder.searchSource();
-        BoolQueryBuilder linkShouldBuilder = QueryBuilders.boolQuery();
-        sourceBuilder.query(linkShouldBuilder);
+    protected SearchBuilder buildQuery(final List<EventQueryCondition> conditionList) {
+        final BoolQueryBuilder query = Query.bool();
+
         conditionList.forEach(condition -> {
-            final BoolQueryBuilder boolQueryBuilder = QueryBuilders.boolQuery();
-            final List<QueryBuilder> mustQueryList = boolQueryBuilder.must();
-            linkShouldBuilder.should(boolQueryBuilder);
-            buildMustQueryListByCondition(condition, mustQueryList);
+            final BoolQueryBuilder bool = Query.bool();
+            query.should(bool);
+            buildMustQueryListByCondition(condition, bool);
         });
         EventQueryCondition condition = conditionList.get(0);
         final Order queryOrder = isNull(condition.getOrder()) ? Order.DES : condition.getOrder();
-        sourceBuilder.sort(Event.START_TIME, Order.DES.equals(queryOrder) ? SortOrder.DESC : SortOrder.ASC);
-
         final PaginationUtils.Page page = PaginationUtils.INSTANCE.exchange(condition.getPaging());
-        sourceBuilder.from(page.getFrom());
-        sourceBuilder.size(page.getLimit());
-        return sourceBuilder;
-    }
 
-    protected SearchSourceBuilder buildQuery(final EventQueryCondition condition) {
-        final SearchSourceBuilder sourceBuilder = SearchSourceBuilder.searchSource();
-        final BoolQueryBuilder boolQueryBuilder = QueryBuilders.boolQuery();
-        sourceBuilder.query(boolQueryBuilder);
+        return Search.builder().query(query)
+                     .sort(
+                         Event.START_TIME,
+                         Order.DES.equals(queryOrder) ? Sort.Order.DESC : Sort.Order.ASC
+                     )
+                     .from(page.getFrom())
+                     .size(page.getLimit());
+    }
 
-        final List<QueryBuilder> mustQueryList = boolQueryBuilder.must();
+    protected SearchBuilder buildQuery(final EventQueryCondition condition) {
+        final BoolQueryBuilder query = Query.bool();
 
-        buildMustQueryListByCondition(condition, mustQueryList);
+        buildMustQueryListByCondition(condition, query);
 
         final Order queryOrder = isNull(condition.getOrder()) ? Order.DES : condition.getOrder();
-        sourceBuilder.sort(Event.START_TIME, Order.DES.equals(queryOrder) ? SortOrder.DESC : SortOrder.ASC);
-
         final PaginationUtils.Page page = PaginationUtils.INSTANCE.exchange(condition.getPaging());
-        sourceBuilder.from(page.getFrom());
-        sourceBuilder.size(page.getLimit());
 
-        return sourceBuilder;
+        return Search.builder()
+                     .query(query)
+                     .sort(
+                         Event.START_TIME,
+                         Order.DES.equals(queryOrder) ? Sort.Order.DESC : Sort.Order.ASC
+                     )
+                     .from(page.getFrom())
+                     .size(page.getLimit());
     }
 
-    protected org.apache.skywalking.oap.server.core.query.type.event.Event parseSearchHit(final SearchHit searchHit) {
-        final org.apache.skywalking.oap.server.core.query.type.event.Event event = new org.apache.skywalking.oap.server.core.query.type.event.Event();
+    protected org.apache.skywalking.oap.server.core.query.type.event.Event parseSearchHit(
+        final SearchHit searchHit) {
+        final org.apache.skywalking.oap.server.core.query.type.event.Event event =
+            new org.apache.skywalking.oap.server.core.query.type.event.Event();
 
-        event.setUuid((String) searchHit.getSourceAsMap().get(Event.UUID));
+        event.setUuid((String) searchHit.getSource().get(Event.UUID));
 
-        String service = searchHit.getSourceAsMap().getOrDefault(Event.SERVICE, "").toString();
-        String serviceInstance = searchHit.getSourceAsMap().getOrDefault(Event.SERVICE_INSTANCE, "").toString();
-        String endpoint = searchHit.getSourceAsMap().getOrDefault(Event.ENDPOINT, "").toString();
+        String service = searchHit.getSource().getOrDefault(Event.SERVICE, "").toString();
+        String serviceInstance =
+            searchHit.getSource().getOrDefault(Event.SERVICE_INSTANCE, "").toString();
+        String endpoint = searchHit.getSource().getOrDefault(Event.ENDPOINT, "").toString();
         event.setSource(new Source(service, serviceInstance, endpoint));
 
-        event.setName((String) searchHit.getSourceAsMap().get(Event.NAME));
-        event.setType(EventType.parse(searchHit.getSourceAsMap().get(Event.TYPE).toString()));
-        event.setMessage((String) searchHit.getSourceAsMap().get(Event.MESSAGE));
-        event.setParameters((String) searchHit.getSourceAsMap().get(Event.PARAMETERS));
-        event.setStartTime(Long.parseLong(searchHit.getSourceAsMap().get(Event.START_TIME).toString()));
-        String endTimeStr = searchHit.getSourceAsMap().getOrDefault(Event.END_TIME, "0").toString();
+        event.setName((String) searchHit.getSource().get(Event.NAME));
+        event.setType(EventType.parse(searchHit.getSource().get(Event.TYPE).toString()));
+        event.setMessage((String) searchHit.getSource().get(Event.MESSAGE));
+        event.setParameters((String) searchHit.getSource().get(Event.PARAMETERS));
+        event.setStartTime(Long.parseLong(searchHit.getSource().get(Event.START_TIME).toString()));
+        String endTimeStr = searchHit.getSource().getOrDefault(Event.END_TIME, "0").toString();
         if (!endTimeStr.isEmpty() && !Objects.equals(endTimeStr, "0")) {
             event.setEndTime(Long.parseLong(endTimeStr));
         }
diff --git a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/LogQueryEsDAO.java b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/LogQueryEsDAO.java
index a60b6f4..eb0ca05 100644
--- a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/LogQueryEsDAO.java
+++ b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/LogQueryEsDAO.java
@@ -18,8 +18,16 @@
 
 package org.apache.skywalking.oap.server.storage.plugin.elasticsearch.query;
 
+import com.google.common.base.Strings;
 import java.io.IOException;
 import java.util.List;
+import org.apache.skywalking.library.elasticsearch.requests.search.BoolQueryBuilder;
+import org.apache.skywalking.library.elasticsearch.requests.search.Query;
+import org.apache.skywalking.library.elasticsearch.requests.search.Search;
+import org.apache.skywalking.library.elasticsearch.requests.search.SearchBuilder;
+import org.apache.skywalking.library.elasticsearch.requests.search.Sort;
+import org.apache.skywalking.library.elasticsearch.response.search.SearchHit;
+import org.apache.skywalking.library.elasticsearch.response.search.SearchResponse;
 import org.apache.skywalking.oap.server.core.analysis.IDManager;
 import org.apache.skywalking.oap.server.core.analysis.manual.log.AbstractLogRecord;
 import org.apache.skywalking.oap.server.core.analysis.manual.log.LogRecord;
@@ -36,14 +44,6 @@ import org.apache.skywalking.oap.server.library.util.CollectionUtils;
 import org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base.EsDAO;
 import org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base.IndexController;
 import org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base.MatchCNameBuilder;
-import org.elasticsearch.action.search.SearchResponse;
-import org.elasticsearch.common.Strings;
-import org.elasticsearch.index.query.BoolQueryBuilder;
-import org.elasticsearch.index.query.QueryBuilder;
-import org.elasticsearch.index.query.QueryBuilders;
-import org.elasticsearch.search.SearchHit;
-import org.elasticsearch.search.builder.SearchSourceBuilder;
-import org.elasticsearch.search.sort.SortOrder;
 
 import static java.util.Objects.nonNull;
 import static org.apache.skywalking.apm.util.StringUtil.isNotEmpty;
@@ -71,46 +71,44 @@ public class LogQueryEsDAO extends EsDAO implements ILogQueryDAO {
                           final List<Tag> tags,
                           final List<String> keywordsOfContent,
                           final List<String> excludingKeywordsOfContent) throws IOException {
-        SearchSourceBuilder sourceBuilder = SearchSourceBuilder.searchSource();
-
-        BoolQueryBuilder boolQueryBuilder = QueryBuilders.boolQuery();
-        sourceBuilder.query(boolQueryBuilder);
-        List<QueryBuilder> mustQueryList = boolQueryBuilder.must();
+        final String index =
+            IndexController.LogicIndicesRegister.getPhysicalTableName(LogRecord.INDEX_NAME);
 
+        final BoolQueryBuilder query = Query.bool();
         if (startSecondTB != 0 && endSecondTB != 0) {
-            mustQueryList.add(QueryBuilders.rangeQuery(Record.TIME_BUCKET).gte(startSecondTB).lte(endSecondTB));
+            query.must(Query.range(Record.TIME_BUCKET).gte(startSecondTB).lte(endSecondTB));
         }
         if (isNotEmpty(serviceId)) {
-            mustQueryList.add(QueryBuilders.termQuery(AbstractLogRecord.SERVICE_ID, serviceId));
+            query.must(Query.term(AbstractLogRecord.SERVICE_ID, serviceId));
         }
         if (isNotEmpty(serviceInstanceId)) {
-            mustQueryList.add(QueryBuilders.termQuery(AbstractLogRecord.SERVICE_INSTANCE_ID, serviceInstanceId));
+            query.must(Query.term(AbstractLogRecord.SERVICE_INSTANCE_ID, serviceInstanceId));
         }
         if (isNotEmpty(endpointId)) {
-            mustQueryList.add(QueryBuilders.termQuery(AbstractLogRecord.ENDPOINT_ID, endpointId));
+            query.must(Query.term(AbstractLogRecord.ENDPOINT_ID, endpointId));
         }
         if (nonNull(relatedTrace)) {
             if (isNotEmpty(relatedTrace.getTraceId())) {
-                mustQueryList.add(QueryBuilders.termQuery(AbstractLogRecord.TRACE_ID, relatedTrace.getTraceId()));
+                query.must(Query.term(AbstractLogRecord.TRACE_ID, relatedTrace.getTraceId()));
             }
             if (isNotEmpty(relatedTrace.getSegmentId())) {
-                mustQueryList.add(
-                    QueryBuilders.termQuery(AbstractLogRecord.TRACE_SEGMENT_ID, relatedTrace.getSegmentId()));
+                query.must(
+                    Query.term(AbstractLogRecord.TRACE_SEGMENT_ID, relatedTrace.getSegmentId()));
             }
             if (nonNull(relatedTrace.getSpanId())) {
-                mustQueryList.add(QueryBuilders.termQuery(AbstractLogRecord.SPAN_ID, relatedTrace.getSpanId()));
+                query.must(Query.term(AbstractLogRecord.SPAN_ID, relatedTrace.getSpanId()));
             }
         }
 
         if (CollectionUtils.isNotEmpty(tags)) {
-            tags.forEach(tag -> mustQueryList.add(QueryBuilders.termQuery(AbstractLogRecord.TAGS, tag.toString())));
+            tags.forEach(tag -> query.must(Query.term(AbstractLogRecord.TAGS, tag.toString())));
         }
 
         if (CollectionUtils.isNotEmpty(keywordsOfContent)) {
             keywordsOfContent.forEach(
                 content ->
-                    mustQueryList.add(
-                        QueryBuilders.matchPhraseQuery(
+                    query.must(
+                        Query.matchPhrase(
                             MatchCNameBuilder.INSTANCE.build(AbstractLogRecord.CONTENT),
                             content
                         )
@@ -121,8 +119,8 @@ public class LogQueryEsDAO extends EsDAO implements ILogQueryDAO {
         if (CollectionUtils.isNotEmpty(excludingKeywordsOfContent)) {
             excludingKeywordsOfContent.forEach(
                 content ->
-                    boolQueryBuilder.mustNot(
-                        QueryBuilders.matchPhraseQuery(
+                    query.mustNot(
+                        Query.matchPhrase(
                             MatchCNameBuilder.INSTANCE.build(AbstractLogRecord.CONTENT),
                             content
                         )
@@ -130,31 +128,41 @@ public class LogQueryEsDAO extends EsDAO implements ILogQueryDAO {
             );
         }
 
-        sourceBuilder.sort(LogRecord.TIMESTAMP, Order.DES.equals(queryOrder) ? SortOrder.DESC : SortOrder.ASC);
-        sourceBuilder.size(limit);
-        sourceBuilder.from(from);
+        final SearchBuilder search =
+            Search.builder().query(query)
+                  .sort(
+                      LogRecord.TIMESTAMP,
+                      Order.DES.equals(queryOrder) ?
+                          Sort.Order.DESC : Sort.Order.ASC
+                  )
+                  .size(limit)
+                  .from(from);
 
-        SearchResponse response = getClient()
-            .search(IndexController.LogicIndicesRegister.getPhysicalTableName(LogRecord.INDEX_NAME), sourceBuilder);
+        SearchResponse response = getClient().search(index, search.build());
 
         Logs logs = new Logs();
-        logs.setTotal((int) response.getHits().totalHits);
+        logs.setTotal(response.getHits().getTotal());
 
         for (SearchHit searchHit : response.getHits().getHits()) {
             Log log = new Log();
-            log.setServiceId((String) searchHit.getSourceAsMap().get(AbstractLogRecord.SERVICE_ID));
-            log.setServiceInstanceId((String) searchHit.getSourceAsMap()
+            log.setServiceId((String) searchHit.getSource().get(AbstractLogRecord.SERVICE_ID));
+            log.setServiceInstanceId((String) searchHit.getSource()
                                                        .get(AbstractLogRecord.SERVICE_INSTANCE_ID));
-            log.setEndpointId((String) searchHit.getSourceAsMap().get(AbstractLogRecord.ENDPOINT_ID));
+            log.setEndpointId(
+                (String) searchHit.getSource().get(AbstractLogRecord.ENDPOINT_ID));
             if (log.getEndpointId() != null) {
-                log.setEndpointName(IDManager.EndpointID.analysisId(log.getEndpointId()).getEndpointName());
+                log.setEndpointName(
+                    IDManager.EndpointID.analysisId(log.getEndpointId()).getEndpointName());
             }
-            log.setTraceId((String) searchHit.getSourceAsMap().get(AbstractLogRecord.TRACE_ID));
-            log.setTimestamp(((Number) searchHit.getSourceAsMap().get(AbstractLogRecord.TIMESTAMP)).longValue());
+            log.setTraceId((String) searchHit.getSource().get(AbstractLogRecord.TRACE_ID));
+            log.setTimestamp(
+                ((Number) searchHit.getSource().get(AbstractLogRecord.TIMESTAMP)).longValue());
             log.setContentType(ContentType.instanceOf(
-                ((Number) searchHit.getSourceAsMap().get(AbstractLogRecord.CONTENT_TYPE)).intValue()));
-            log.setContent((String) searchHit.getSourceAsMap().get(AbstractLogRecord.CONTENT));
-            String dataBinaryBase64 = (String) searchHit.getSourceAsMap().get(AbstractLogRecord.TAGS_RAW_DATA);
+                ((Number) searchHit.getSource()
+                                   .get(AbstractLogRecord.CONTENT_TYPE)).intValue()));
+            log.setContent((String) searchHit.getSource().get(AbstractLogRecord.CONTENT));
+            String dataBinaryBase64 =
+                (String) searchHit.getSource().get(AbstractLogRecord.TAGS_RAW_DATA);
             if (!Strings.isNullOrEmpty(dataBinaryBase64)) {
                 parserDataBinary(dataBinaryBase64, log.getTags());
             }
diff --git a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/MetadataQueryEsDAO.java b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/MetadataQueryEsDAO.java
index 8464d3b..b34e971 100644
--- a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/MetadataQueryEsDAO.java
+++ b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/MetadataQueryEsDAO.java
@@ -25,7 +25,14 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
+import java.util.stream.Collectors;
 import org.apache.skywalking.apm.util.StringUtil;
+import org.apache.skywalking.library.elasticsearch.requests.search.BoolQueryBuilder;
+import org.apache.skywalking.library.elasticsearch.requests.search.Query;
+import org.apache.skywalking.library.elasticsearch.requests.search.Search;
+import org.apache.skywalking.library.elasticsearch.requests.search.SearchBuilder;
+import org.apache.skywalking.library.elasticsearch.response.search.SearchHit;
+import org.apache.skywalking.library.elasticsearch.response.search.SearchResponse;
 import org.apache.skywalking.oap.server.core.analysis.NodeType;
 import org.apache.skywalking.oap.server.core.analysis.TimeBucket;
 import org.apache.skywalking.oap.server.core.analysis.manual.endpoint.EndpointTraffic;
@@ -42,11 +49,6 @@ import org.apache.skywalking.oap.server.library.client.elasticsearch.ElasticSear
 import org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base.EsDAO;
 import org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base.IndexController;
 import org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base.MatchCNameBuilder;
-import org.elasticsearch.action.search.SearchResponse;
-import org.elasticsearch.index.query.BoolQueryBuilder;
-import org.elasticsearch.index.query.QueryBuilders;
-import org.elasticsearch.search.SearchHit;
-import org.elasticsearch.search.builder.SearchSourceBuilder;
 
 import static org.apache.skywalking.oap.server.core.analysis.manual.instance.InstanceTraffic.PropertyUtil.LANGUAGE;
 
@@ -60,120 +62,111 @@ public class MetadataQueryEsDAO extends EsDAO implements IMetadataQueryDAO {
 
     @Override
     public List<Service> getAllServices(final String group) throws IOException {
-        SearchSourceBuilder sourceBuilder = SearchSourceBuilder.searchSource();
+        final String index =
+            IndexController.LogicIndicesRegister.getPhysicalTableName(ServiceTraffic.INDEX_NAME);
 
-        BoolQueryBuilder boolQueryBuilder = QueryBuilders.boolQuery();
-        boolQueryBuilder.must().add(QueryBuilders.termQuery(ServiceTraffic.NODE_TYPE, NodeType.Normal.value()));
+        final BoolQueryBuilder query =
+            Query.bool()
+                 .must(Query.term(ServiceTraffic.NODE_TYPE, NodeType.Normal.value()));
+        final SearchBuilder search = Search.builder().query(query).size(queryMaxSize);
         if (StringUtil.isNotEmpty(group)) {
-            boolQueryBuilder.must().add(QueryBuilders.termQuery(ServiceTraffic.GROUP, group));
+            query.must(Query.term(ServiceTraffic.GROUP, group));
         }
+        final SearchResponse results = getClient().search(index, search.build());
 
-        sourceBuilder.query(boolQueryBuilder);
-        sourceBuilder.size(queryMaxSize);
-
-        SearchResponse response = getClient()
-            .search(IndexController.LogicIndicesRegister.getPhysicalTableName(ServiceTraffic.INDEX_NAME), sourceBuilder);
-
-        return buildServices(response);
+        return buildServices(results);
     }
 
     @Override
     public List<Service> getAllBrowserServices() throws IOException {
-        SearchSourceBuilder sourceBuilder = SearchSourceBuilder.searchSource();
-
-        BoolQueryBuilder boolQueryBuilder = QueryBuilders.boolQuery();
-        boolQueryBuilder.must().add(QueryBuilders.termQuery(ServiceTraffic.NODE_TYPE, NodeType.Browser.value()));
-
-        sourceBuilder.query(boolQueryBuilder);
-        sourceBuilder.size(queryMaxSize);
-
-        SearchResponse response = getClient()
-            .search(IndexController.LogicIndicesRegister.getPhysicalTableName(ServiceTraffic.INDEX_NAME), sourceBuilder);
-
-        return buildServices(response);
+        final String index =
+            IndexController.LogicIndicesRegister.getPhysicalTableName(ServiceTraffic.INDEX_NAME);
+        final BoolQueryBuilder query = Query.bool().must(
+            Query.term(ServiceTraffic.NODE_TYPE, NodeType.Browser.value()));
+        final SearchBuilder search = Search.builder().query(query).size(queryMaxSize);
+        final SearchResponse result = getClient().search(index, search.build());
+
+        return buildServices(result);
     }
 
     @Override
     public List<Database> getAllDatabases() throws IOException {
-        SearchSourceBuilder sourceBuilder = SearchSourceBuilder.searchSource();
-
-        BoolQueryBuilder boolQueryBuilder = QueryBuilders.boolQuery();
-        boolQueryBuilder.must().add(QueryBuilders.termQuery(ServiceTraffic.NODE_TYPE, NodeType.Database.value()));
+        final String index =
+            IndexController.LogicIndicesRegister.getPhysicalTableName(ServiceTraffic.INDEX_NAME);
 
-        sourceBuilder.query(boolQueryBuilder);
-        sourceBuilder.size(queryMaxSize);
+        final BoolQueryBuilder query = Query.bool().must(
+            Query.term(ServiceTraffic.NODE_TYPE, NodeType.Database.value()));
+        final SearchBuilder search = Search.builder().query(query).size(queryMaxSize);
+        final SearchResponse results = getClient().search(index, search.build());
 
-        SearchResponse response = getClient()
-            .search(IndexController.LogicIndicesRegister.getPhysicalTableName(ServiceTraffic.INDEX_NAME), sourceBuilder);
-
-        final List<Service> serviceList = buildServices(response);
-        List<Database> databases = new ArrayList<>();
-        for (Service service : serviceList) {
+        final List<Service> serviceList = buildServices(results);
+        return serviceList.stream().map(service -> {
             Database database = new Database();
             database.setId(service.getId());
             database.setName(service.getName());
-            databases.add(database);
-        }
-        return databases;
+            return database;
+        }).collect(Collectors.toList());
     }
 
     @Override
     public List<Service> searchServices(String keyword) throws IOException {
-        SearchSourceBuilder sourceBuilder = SearchSourceBuilder.searchSource();
+        final String index =
+            IndexController.LogicIndicesRegister.getPhysicalTableName(ServiceTraffic.INDEX_NAME);
 
-        BoolQueryBuilder boolQueryBuilder = QueryBuilders.boolQuery();
+        final BoolQueryBuilder query =
+            Query.bool()
+                 .must(Query.term(ServiceTraffic.NODE_TYPE, NodeType.Normal.value()));
+        final SearchBuilder search = Search.builder().query(query).size(queryMaxSize);
 
-        boolQueryBuilder.must().add(QueryBuilders.termQuery(ServiceTraffic.NODE_TYPE, NodeType.Normal.value()));
         if (!Strings.isNullOrEmpty(keyword)) {
             String matchCName = MatchCNameBuilder.INSTANCE.build(ServiceTraffic.NAME);
-            boolQueryBuilder.must().add(QueryBuilders.matchQuery(matchCName, keyword));
+            query.must(Query.match(matchCName, keyword));
         }
 
-        sourceBuilder.query(boolQueryBuilder);
-        sourceBuilder.size(queryMaxSize);
-
-        SearchResponse response = getClient()
-            .search(IndexController.LogicIndicesRegister.getPhysicalTableName(ServiceTraffic.INDEX_NAME), sourceBuilder);
+        SearchResponse response = getClient().search(index, search.build());
         return buildServices(response);
     }
 
     @Override
     public Service searchService(String serviceCode) throws IOException {
-        SearchSourceBuilder sourceBuilder = SearchSourceBuilder.searchSource();
-        BoolQueryBuilder boolQueryBuilder = QueryBuilders.boolQuery();
-        boolQueryBuilder.must().add(QueryBuilders.termQuery(ServiceTraffic.NODE_TYPE, NodeType.Normal.value()));
-        boolQueryBuilder.must().add(QueryBuilders.termQuery(ServiceTraffic.NAME, serviceCode));
-        sourceBuilder.query(boolQueryBuilder);
-        sourceBuilder.size(1);
-        SearchResponse response = getClient()
-            .search(IndexController.LogicIndicesRegister.getPhysicalTableName(ServiceTraffic.INDEX_NAME), sourceBuilder);
+        final String index =
+            IndexController.LogicIndicesRegister.getPhysicalTableName(ServiceTraffic.INDEX_NAME);
+        final BoolQueryBuilder query =
+            Query.bool()
+                 .must(Query.term(ServiceTraffic.NODE_TYPE, NodeType.Normal.value()))
+                 .must(Query.term(ServiceTraffic.NAME, serviceCode));
+        final SearchBuilder search = Search.builder().query(query).size(1);
+
+        final SearchResponse response = getClient().search(index, search.build());
         final List<Service> services = buildServices(response);
         return services.size() > 0 ? services.get(0) : null;
     }
 
     @Override
-    public List<Endpoint> searchEndpoint(String keyword, String serviceId, int limit) throws IOException {
-        SearchSourceBuilder sourceBuilder = SearchSourceBuilder.searchSource();
+    public List<Endpoint> searchEndpoint(String keyword, String serviceId, int limit)
+        throws IOException {
+        final String index = IndexController.LogicIndicesRegister.getPhysicalTableName(
+            EndpointTraffic.INDEX_NAME);
 
-        BoolQueryBuilder boolQueryBuilder = QueryBuilders.boolQuery();
-        boolQueryBuilder.must().add(QueryBuilders.termQuery(EndpointTraffic.SERVICE_ID, serviceId));
+        final BoolQueryBuilder query =
+            Query.bool()
+                 .must(Query.term(EndpointTraffic.SERVICE_ID, serviceId));
 
         if (!Strings.isNullOrEmpty(keyword)) {
             String matchCName = MatchCNameBuilder.INSTANCE.build(EndpointTraffic.NAME);
-            boolQueryBuilder.must().add(QueryBuilders.matchQuery(matchCName, keyword));
+            query.must(Query.match(matchCName, keyword));
         }
 
-        sourceBuilder.query(boolQueryBuilder);
-        sourceBuilder.size(limit);
+        final SearchBuilder search = Search.builder().query(query).size(limit);
 
-        SearchResponse response = getClient()
-            .search(IndexController.LogicIndicesRegister.getPhysicalTableName(EndpointTraffic.INDEX_NAME), sourceBuilder);
+        final SearchResponse response = getClient().search(index, search.build());
 
         List<Endpoint> endpoints = new ArrayList<>();
         for (SearchHit searchHit : response.getHits()) {
-            Map<String, Object> sourceAsMap = searchHit.getSourceAsMap();
+            Map<String, Object> sourceAsMap = searchHit.getSource();
 
-            final EndpointTraffic endpointTraffic = new EndpointTraffic.Builder().storage2Entity(sourceAsMap);
+            final EndpointTraffic endpointTraffic =
+                new EndpointTraffic.Builder().storage2Entity(sourceAsMap);
 
             Endpoint endpoint = new Endpoint();
             endpoint.setId(endpointTraffic.id());
@@ -187,27 +180,24 @@ public class MetadataQueryEsDAO extends EsDAO implements IMetadataQueryDAO {
     @Override
     public List<ServiceInstance> getServiceInstances(long startTimestamp, long endTimestamp,
                                                      String serviceId) throws IOException {
-        SearchSourceBuilder sourceBuilder = SearchSourceBuilder.searchSource();
-
-        BoolQueryBuilder boolQueryBuilder = QueryBuilders.boolQuery();
+        final String index =
+            IndexController.LogicIndicesRegister.getPhysicalTableName(InstanceTraffic.INDEX_NAME);
 
         final long minuteTimeBucket = TimeBucket.getMinuteTimeBucket(startTimestamp);
+        final BoolQueryBuilder query =
+            Query.bool()
+                 .must(Query.range(InstanceTraffic.LAST_PING_TIME_BUCKET).gte(minuteTimeBucket))
+                 .must(Query.term(InstanceTraffic.SERVICE_ID, serviceId));
+        final SearchBuilder search = Search.builder().query(query).size(queryMaxSize);
 
-        boolQueryBuilder.must()
-                        .add(QueryBuilders.rangeQuery(InstanceTraffic.LAST_PING_TIME_BUCKET).gte(minuteTimeBucket));
-        boolQueryBuilder.must().add(QueryBuilders.termQuery(InstanceTraffic.SERVICE_ID, serviceId));
-
-        sourceBuilder.query(boolQueryBuilder);
-        sourceBuilder.size(queryMaxSize);
-
-        SearchResponse response = getClient()
-            .search(IndexController.LogicIndicesRegister.getPhysicalTableName(InstanceTraffic.INDEX_NAME), sourceBuilder);
+        final SearchResponse response = getClient().search(index, search.build());
 
         List<ServiceInstance> serviceInstances = new ArrayList<>();
         for (SearchHit searchHit : response.getHits()) {
-            Map<String, Object> sourceAsMap = searchHit.getSourceAsMap();
+            Map<String, Object> sourceAsMap = searchHit.getSource();
 
-            final InstanceTraffic instanceTraffic = new InstanceTraffic.Builder().storage2Entity(sourceAsMap);
+            final InstanceTraffic instanceTraffic =
+                new InstanceTraffic.Builder().storage2Entity(sourceAsMap);
 
             ServiceInstance serviceInstance = new ServiceInstance();
             serviceInstance.setId(instanceTraffic.id());
@@ -235,9 +225,8 @@ public class MetadataQueryEsDAO extends EsDAO implements IMetadataQueryDAO {
 
     private List<Service> buildServices(SearchResponse response) {
         List<Service> services = new ArrayList<>();
-        for (SearchHit searchHit : response.getHits()) {
-            Map<String, Object> sourceAsMap = searchHit.getSourceAsMap();
-
+        for (SearchHit hit : response.getHits()) {
+            final Map<String, Object> sourceAsMap = hit.getSource();
             final ServiceTraffic.Builder builder = new ServiceTraffic.Builder();
             final ServiceTraffic serviceTraffic = builder.storage2Entity(sourceAsMap);
 
diff --git a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/MetricsQueryEsDAO.java b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/MetricsQueryEsDAO.java
index 33d963f..50caad0 100644
--- a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/MetricsQueryEsDAO.java
+++ b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/MetricsQueryEsDAO.java
@@ -20,9 +20,20 @@ package org.apache.skywalking.oap.server.storage.plugin.elasticsearch.query;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
+import org.apache.skywalking.library.elasticsearch.requests.search.Query;
+import org.apache.skywalking.library.elasticsearch.requests.search.RangeQueryBuilder;
+import org.apache.skywalking.library.elasticsearch.requests.search.Search;
+import org.apache.skywalking.library.elasticsearch.requests.search.SearchBuilder;
+import org.apache.skywalking.library.elasticsearch.requests.search.aggregation.Aggregation;
+import org.apache.skywalking.library.elasticsearch.requests.search.aggregation.TermsAggregationBuilder;
+import org.apache.skywalking.library.elasticsearch.response.Document;
+import org.apache.skywalking.library.elasticsearch.response.Documents;
+import org.apache.skywalking.library.elasticsearch.response.search.SearchResponse;
 import org.apache.skywalking.oap.server.core.analysis.metrics.DataTable;
 import org.apache.skywalking.oap.server.core.analysis.metrics.HistogramMetrics;
 import org.apache.skywalking.oap.server.core.analysis.metrics.Metrics;
@@ -39,17 +50,6 @@ import org.apache.skywalking.oap.server.core.storage.query.IMetricsQueryDAO;
 import org.apache.skywalking.oap.server.library.client.elasticsearch.ElasticSearchClient;
 import org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base.EsDAO;
 import org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base.IndexController;
-import org.elasticsearch.action.search.SearchResponse;
-import org.elasticsearch.index.query.BoolQueryBuilder;
-import org.elasticsearch.index.query.QueryBuilders;
-import org.elasticsearch.index.query.RangeQueryBuilder;
-import org.elasticsearch.search.SearchHit;
-import org.elasticsearch.search.aggregations.AggregationBuilders;
-import org.elasticsearch.search.aggregations.bucket.terms.Terms;
-import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder;
-import org.elasticsearch.search.aggregations.metrics.avg.Avg;
-import org.elasticsearch.search.aggregations.metrics.sum.Sum;
-import org.elasticsearch.search.builder.SearchSourceBuilder;
 
 public class MetricsQueryEsDAO extends EsDAO implements IMetricsQueryDAO {
 
@@ -61,37 +61,35 @@ public class MetricsQueryEsDAO extends EsDAO implements IMetricsQueryDAO {
     public long readMetricsValue(final MetricsCondition condition,
                                  final String valueColumnName,
                                  final Duration duration) throws IOException {
-        SearchSourceBuilder sourceBuilder = SearchSourceBuilder.searchSource();
-        buildQuery(sourceBuilder, condition, duration);
+        final SearchBuilder sourceBuilder = buildQuery(condition, duration);
         int defaultValue = ValueColumnMetadata.INSTANCE.getDefaultValue(condition.getName());
-        final Function function = ValueColumnMetadata.INSTANCE.getValueFunction(condition.getName());
+        Function function = ValueColumnMetadata.INSTANCE.getValueFunction(condition.getName());
         if (function == Function.Latest) {
-            return readMetricsValues(condition, valueColumnName, duration).getValues().latestValue(defaultValue);
+            return readMetricsValues(condition, valueColumnName, duration)
+                .getValues().latestValue(defaultValue);
         }
 
-        TermsAggregationBuilder entityIdAggregation = AggregationBuilders.terms(Metrics.ENTITY_ID)
-                                                                         .field(Metrics.ENTITY_ID)
-                                                                         .size(1);
+        final TermsAggregationBuilder entityIdAggregation =
+            Aggregation.terms(Metrics.ENTITY_ID)
+                       .field(Metrics.ENTITY_ID)
+                       .size(1);
         functionAggregation(function, entityIdAggregation, valueColumnName);
 
         sourceBuilder.aggregation(entityIdAggregation);
 
-        SearchResponse response = getClient()
-            .search(IndexController.LogicIndicesRegister.getPhysicalTableName(condition.getName()), sourceBuilder);
-
-        Terms idTerms = response.getAggregations().get(Metrics.ENTITY_ID);
-        for (Terms.Bucket idBucket : idTerms.getBuckets()) {
-            switch (function) {
-                case Sum:
-                    Sum sum = idBucket.getAggregations().get(valueColumnName);
-                    return (long) sum.getValue();
-                case Avg:
-                    Avg avg = idBucket.getAggregations().get(valueColumnName);
-                    return (long) avg.getValue();
-                default:
-                    avg = idBucket.getAggregations().get(valueColumnName);
-                    return (long) avg.getValue();
-            }
+        final String index =
+            IndexController.LogicIndicesRegister.getPhysicalTableName(condition.getName());
+
+        final SearchResponse response = getClient().search(index, sourceBuilder.build());
+
+        final Map<String, Object> idTerms =
+            (Map<String, Object>) response.getAggregations().get(Metrics.ENTITY_ID);
+        final List<Map<String, Object>> buckets =
+            (List<Map<String, Object>>) idTerms.get("buckets");
+
+        for (Map<String, Object> idBucket : buckets) {
+            final Map<String, Object> agg = (Map<String, Object>) idBucket.get(valueColumnName);
+            return ((Number) agg.get("value")).longValue();
         }
         return defaultValue;
     }
@@ -100,7 +98,8 @@ public class MetricsQueryEsDAO extends EsDAO implements IMetricsQueryDAO {
     public MetricsValues readMetricsValues(final MetricsCondition condition,
                                            final String valueColumnName,
                                            final Duration duration) throws IOException {
-        String tableName = IndexController.LogicIndicesRegister.getPhysicalTableName(condition.getName());
+        String tableName =
+            IndexController.LogicIndicesRegister.getPhysicalTableName(condition.getName());
         final List<PointOfTime> pointOfTimes = duration.assembleDurationPoints();
         List<String> ids = new ArrayList<>(pointOfTimes.size());
 
@@ -111,12 +110,15 @@ public class MetricsQueryEsDAO extends EsDAO implements IMetricsQueryDAO {
             }
             ids.add(id);
         });
+        MetricsValues metricsValues = new MetricsValues();
 
-        SearchResponse response = getClient()
-            .ids(tableName, ids.toArray(new String[0]));
-        Map<String, Map<String, Object>> idMap = toMap(response);
+        Optional<Documents> response = getClient().ids(tableName, ids);
+        if (!response.isPresent()) {
+            return metricsValues;
+        }
+
+        Map<String, Map<String, Object>> idMap = toMap(response.get());
 
-        MetricsValues metricsValues = new MetricsValues();
         // Label is null, because in readMetricsValues, no label parameter.
         IntValues intValues = metricsValues.getValues();
         for (String id : ids) {
@@ -133,7 +135,8 @@ public class MetricsQueryEsDAO extends EsDAO implements IMetricsQueryDAO {
         }
 
         metricsValues.setValues(
-            Util.sortValues(intValues, ids, ValueColumnMetadata.INSTANCE.getDefaultValue(condition.getName()))
+            Util.sortValues(
+                intValues, ids, ValueColumnMetadata.INSTANCE.getDefaultValue(condition.getName()))
         );
 
         return metricsValues;
@@ -143,9 +146,11 @@ public class MetricsQueryEsDAO extends EsDAO implements IMetricsQueryDAO {
     public List<MetricsValues> readLabeledMetricsValues(final MetricsCondition condition,
                                                         final String valueColumnName,
                                                         final List<String> labels,
-                                                        final Duration duration) throws IOException {
+                                                        final Duration duration)
+        throws IOException {
         final List<PointOfTime> pointOfTimes = duration.assembleDurationPoints();
-        String tableName = IndexController.LogicIndicesRegister.getPhysicalTableName(condition.getName());
+        String tableName =
+            IndexController.LogicIndicesRegister.getPhysicalTableName(condition.getName());
         boolean aggregationMode = !tableName.equals(condition.getName());
         List<String> ids = new ArrayList<>(pointOfTimes.size());
         pointOfTimes.forEach(pointOfTime -> {
@@ -156,11 +161,16 @@ public class MetricsQueryEsDAO extends EsDAO implements IMetricsQueryDAO {
             ids.add(id);
         });
 
-        SearchResponse response = getClient().ids(tableName, ids.toArray(new String[0]));
+        Optional<Documents> response = getClient().ids(tableName, ids);
+        if (!response.isPresent()) {
+            return Collections.emptyList();
+        }
         Map<String, DataTable> idMap = new HashMap<>();
-        SearchHit[] hits = response.getHits().getHits();
-        for (SearchHit hit : hits) {
-            idMap.put(hit.getId(), new DataTable((String) hit.getSourceAsMap().getOrDefault(valueColumnName, "")));
+        for (final Document document : response.get()) {
+            idMap.put(
+                document.getId(),
+                new DataTable((String) document.getSource().getOrDefault(valueColumnName, ""))
+            );
         }
         return Util.composeLabelValue(condition, labels, ids, idMap);
     }
@@ -170,7 +180,8 @@ public class MetricsQueryEsDAO extends EsDAO implements IMetricsQueryDAO {
                                final String valueColumnName,
                                final Duration duration) throws IOException {
         final List<PointOfTime> pointOfTimes = duration.assembleDurationPoints();
-        String tableName = IndexController.LogicIndicesRegister.getPhysicalTableName(condition.getName());
+        String tableName =
+            IndexController.LogicIndicesRegister.getPhysicalTableName(condition.getName());
         boolean aggregationMode = !tableName.equals(condition.getName());
         List<String> ids = new ArrayList<>(pointOfTimes.size());
         pointOfTimes.forEach(pointOfTime -> {
@@ -181,11 +192,14 @@ public class MetricsQueryEsDAO extends EsDAO implements IMetricsQueryDAO {
             ids.add(id);
         });
 
-        SearchResponse response = getClient().ids(tableName, ids.toArray(new String[0]));
-        Map<String, Map<String, Object>> idMap = toMap(response);
-
         HeatMap heatMap = new HeatMap();
 
+        Optional<Documents> response = getClient().ids(tableName, ids);
+        if (!response.isPresent()) {
+            return heatMap;
+        }
+        Map<String, Map<String, Object>> idMap = toMap(response.get());
+
         final int defaultValue = ValueColumnMetadata.INSTANCE.getDefaultValue(condition.getName());
         for (String id : ids) {
             Map<String, Object> source = idMap.get(id);
@@ -200,65 +214,70 @@ public class MetricsQueryEsDAO extends EsDAO implements IMetricsQueryDAO {
         return heatMap;
     }
 
-    protected void functionAggregation(Function function, TermsAggregationBuilder parentAggBuilder, String valueCName) {
+    protected void functionAggregation(Function function,
+                                       TermsAggregationBuilder parentAggBuilder,
+                                       String valueCName) {
         switch (function) {
             case Avg:
-                parentAggBuilder.subAggregation(AggregationBuilders.avg(valueCName).field(valueCName));
+                parentAggBuilder.subAggregation(Aggregation.avg(valueCName).field(valueCName));
                 break;
             case Sum:
-                parentAggBuilder.subAggregation(AggregationBuilders.sum(valueCName).field(valueCName));
+                parentAggBuilder.subAggregation(Aggregation.sum(valueCName).field(valueCName));
                 break;
             default:
-                parentAggBuilder.subAggregation(AggregationBuilders.avg(valueCName).field(valueCName));
+                parentAggBuilder.subAggregation(Aggregation.avg(valueCName).field(valueCName));
                 break;
         }
     }
 
-    protected final void buildQuery(SearchSourceBuilder sourceBuilder, MetricsCondition condition, Duration duration) {
-        final List<PointOfTime> pointOfTimes = duration.assembleDurationPoints();
-        List<String> ids = new ArrayList<>(pointOfTimes.size());
-        pointOfTimes.forEach(pointOfTime -> {
-            ids.add(pointOfTime.id(condition.getEntity().buildId()));
-        });
+    protected final SearchBuilder buildQuery(MetricsCondition condition, Duration duration) {
+        final SearchBuilder sourceBuilder = Search.builder();
 
-        RangeQueryBuilder rangeQueryBuilder = QueryBuilders.rangeQuery(Metrics.TIME_BUCKET)
-                                                           .gte(duration.getStartTimeBucket())
-                                                           .lte(duration.getEndTimeBucket());
+        final RangeQueryBuilder rangeQueryBuilder =
+            Query.range(Metrics.TIME_BUCKET)
+                 .gte(duration.getStartTimeBucket())
+                 .lte(duration.getEndTimeBucket());
 
         final String entityId = condition.getEntity().buildId();
 
-        if (entityId == null && IndexController.LogicIndicesRegister.isMetricTable(condition.getName())) {
-            BoolQueryBuilder boolQuery = QueryBuilders.boolQuery();
-            boolQuery.must().add(rangeQueryBuilder);
-            boolQuery.must().add(QueryBuilders.termQuery(
-                IndexController.LogicIndicesRegister.METRIC_TABLE_NAME,
-                condition.getName()
-            ));
+        if (entityId == null &&
+            IndexController.LogicIndicesRegister.isMetricTable(condition.getName())) {
+            sourceBuilder.query(
+                Query.bool()
+                     .must(rangeQueryBuilder)
+                     .must(Query.term(
+                         IndexController.LogicIndicesRegister.METRIC_TABLE_NAME,
+                         condition.getName()
+                     ))
+            );
         } else if (entityId == null) {
             sourceBuilder.query(rangeQueryBuilder);
         } else if (IndexController.LogicIndicesRegister.isMetricTable(condition.getName())) {
-            BoolQueryBuilder boolQuery = QueryBuilders.boolQuery();
-            boolQuery.must().add(rangeQueryBuilder);
-            boolQuery.must().add(QueryBuilders.termsQuery(Metrics.ENTITY_ID, entityId));
-            boolQuery.must().add(QueryBuilders.termQuery(
-                IndexController.LogicIndicesRegister.METRIC_TABLE_NAME,
-                condition.getName()
-            ));
-            sourceBuilder.query(boolQuery);
+            sourceBuilder.query(
+                Query.bool()
+                     .must(rangeQueryBuilder)
+                     .must(Query.term(Metrics.ENTITY_ID, entityId))
+                     .must(Query.term(
+                         IndexController.LogicIndicesRegister.METRIC_TABLE_NAME,
+                         condition.getName()
+                     ))
+            );
         } else {
-            BoolQueryBuilder boolQuery = QueryBuilders.boolQuery();
-            boolQuery.must().add(rangeQueryBuilder);
-            boolQuery.must().add(QueryBuilders.termsQuery(Metrics.ENTITY_ID, entityId));
-            sourceBuilder.query(boolQuery);
+            sourceBuilder.query(
+                Query.bool()
+                     .must(rangeQueryBuilder)
+                     .must(Query.term(Metrics.ENTITY_ID, entityId))
+            );
         }
         sourceBuilder.size(0);
+
+        return sourceBuilder;
     }
 
-    private Map<String, Map<String, Object>> toMap(SearchResponse response) {
+    private Map<String, Map<String, Object>> toMap(Documents documents) {
         Map<String, Map<String, Object>> result = new HashMap<>();
-        SearchHit[] hits = response.getHits().getHits();
-        for (SearchHit hit : hits) {
-            result.put(hit.getId(), hit.getSourceAsMap());
+        for (final Document document : documents) {
+            result.put(document.getId(), document.getSource());
         }
         return result;
     }
diff --git a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/ProfileTaskLogEsDAO.java b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/ProfileTaskLogEsDAO.java
index 0208b4c..9a890bd 100644
--- a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/ProfileTaskLogEsDAO.java
+++ b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/ProfileTaskLogEsDAO.java
@@ -21,6 +21,12 @@ package org.apache.skywalking.oap.server.storage.plugin.elasticsearch.query;
 import java.io.IOException;
 import java.util.LinkedList;
 import java.util.List;
+import org.apache.skywalking.library.elasticsearch.requests.search.Query;
+import org.apache.skywalking.library.elasticsearch.requests.search.Search;
+import org.apache.skywalking.library.elasticsearch.requests.search.SearchBuilder;
+import org.apache.skywalking.library.elasticsearch.requests.search.Sort;
+import org.apache.skywalking.library.elasticsearch.response.search.SearchHit;
+import org.apache.skywalking.library.elasticsearch.response.search.SearchResponse;
 import org.apache.skywalking.oap.server.core.profile.ProfileTaskLogRecord;
 import org.apache.skywalking.oap.server.core.query.type.ProfileTaskLog;
 import org.apache.skywalking.oap.server.core.query.type.ProfileTaskLogOperationType;
@@ -28,12 +34,6 @@ import org.apache.skywalking.oap.server.core.storage.profile.IProfileTaskLogQuer
 import org.apache.skywalking.oap.server.library.client.elasticsearch.ElasticSearchClient;
 import org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base.EsDAO;
 import org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base.IndexController;
-import org.elasticsearch.action.search.SearchResponse;
-import org.elasticsearch.index.query.BoolQueryBuilder;
-import org.elasticsearch.index.query.QueryBuilders;
-import org.elasticsearch.search.SearchHit;
-import org.elasticsearch.search.builder.SearchSourceBuilder;
-import org.elasticsearch.search.sort.SortOrder;
 
 public class ProfileTaskLogEsDAO extends EsDAO implements IProfileTaskLogQueryDAO {
     private final int queryMaxSize;
@@ -46,16 +46,14 @@ public class ProfileTaskLogEsDAO extends EsDAO implements IProfileTaskLogQueryDA
 
     @Override
     public List<ProfileTaskLog> getTaskLogList() throws IOException {
-        final SearchSourceBuilder sourceBuilder = SearchSourceBuilder.searchSource();
+        final String index = IndexController.LogicIndicesRegister.getPhysicalTableName(
+            ProfileTaskLogRecord.INDEX_NAME);
+        final SearchBuilder search =
+            Search.builder().query(Query.bool())
+                  .sort(ProfileTaskLogRecord.OPERATION_TIME, Sort.Order.DESC)
+                  .size(queryMaxSize);
 
-        final BoolQueryBuilder boolQueryBuilder = QueryBuilders.boolQuery();
-        sourceBuilder.query(boolQueryBuilder);
-
-        sourceBuilder.sort(ProfileTaskLogRecord.OPERATION_TIME, SortOrder.DESC);
-        sourceBuilder.size(queryMaxSize);
-
-        final SearchResponse response = getClient().search(
-            IndexController.LogicIndicesRegister.getPhysicalTableName(ProfileTaskLogRecord.INDEX_NAME), sourceBuilder);
+        final SearchResponse response = getClient().search(index, search.build());
 
         final LinkedList<ProfileTaskLog> tasks = new LinkedList<>();
         for (SearchHit searchHit : response.getHits().getHits()) {
@@ -68,12 +66,16 @@ public class ProfileTaskLogEsDAO extends EsDAO implements IProfileTaskLogQueryDA
     private ProfileTaskLog parseTaskLog(SearchHit data) {
         return ProfileTaskLog.builder()
                              .id(data.getId())
-                             .taskId((String) data.getSourceAsMap().get(ProfileTaskLogRecord.TASK_ID))
-                             .instanceId((String) data.getSourceAsMap().get(ProfileTaskLogRecord.INSTANCE_ID))
+                             .taskId(
+                                 (String) data.getSource().get(ProfileTaskLogRecord.TASK_ID))
+                             .instanceId(
+                                 (String) data.getSource().get(ProfileTaskLogRecord.INSTANCE_ID))
                              .operationType(ProfileTaskLogOperationType.parse(
-                                 ((Number) data.getSourceAsMap().get(ProfileTaskLogRecord.OPERATION_TYPE)).intValue()))
+                                 ((Number) data.getSource().get(
+                                     ProfileTaskLogRecord.OPERATION_TYPE)).intValue()))
                              .operationTime(
-                                 ((Number) data.getSourceAsMap().get(ProfileTaskLogRecord.OPERATION_TIME)).longValue())
+                                 ((Number) data.getSource().get(
+                                     ProfileTaskLogRecord.OPERATION_TIME)).longValue())
                              .build();
     }
 }
diff --git a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/ProfileTaskQueryEsDAO.java b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/ProfileTaskQueryEsDAO.java
index ac9705a..4384175 100644
--- a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/ProfileTaskQueryEsDAO.java
+++ b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/ProfileTaskQueryEsDAO.java
@@ -21,19 +21,21 @@ package org.apache.skywalking.oap.server.storage.plugin.elasticsearch.query;
 import java.io.IOException;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map;
 import org.apache.skywalking.apm.util.StringUtil;
+import org.apache.skywalking.library.elasticsearch.requests.search.BoolQueryBuilder;
+import org.apache.skywalking.library.elasticsearch.requests.search.Query;
+import org.apache.skywalking.library.elasticsearch.requests.search.Search;
+import org.apache.skywalking.library.elasticsearch.requests.search.SearchBuilder;
+import org.apache.skywalking.library.elasticsearch.requests.search.Sort;
+import org.apache.skywalking.library.elasticsearch.response.search.SearchHit;
+import org.apache.skywalking.library.elasticsearch.response.search.SearchResponse;
 import org.apache.skywalking.oap.server.core.profile.ProfileTaskRecord;
 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.client.elasticsearch.ElasticSearchClient;
 import org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base.EsDAO;
 import org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base.IndexController;
-import org.elasticsearch.action.search.SearchResponse;
-import org.elasticsearch.index.query.BoolQueryBuilder;
-import org.elasticsearch.index.query.QueryBuilders;
-import org.elasticsearch.search.SearchHit;
-import org.elasticsearch.search.builder.SearchSourceBuilder;
-import org.elasticsearch.search.sort.SortOrder;
 
 public class ProfileTaskQueryEsDAO extends EsDAO implements IProfileTaskQueryDAO {
 
@@ -45,40 +47,40 @@ public class ProfileTaskQueryEsDAO extends EsDAO implements IProfileTaskQueryDAO
     }
 
     @Override
-    public List<ProfileTask> getTaskList(String serviceId, String endpointName, Long startTimeBucket,
-                                         Long endTimeBucket, Integer limit) throws IOException {
-        SearchSourceBuilder sourceBuilder = SearchSourceBuilder.searchSource();
-
-        final BoolQueryBuilder boolQueryBuilder = QueryBuilders.boolQuery();
-        sourceBuilder.query(boolQueryBuilder);
+    public List<ProfileTask> getTaskList(String serviceId, String endpointName,
+                                         Long startTimeBucket, Long endTimeBucket,
+                                         Integer limit) throws IOException {
+        final String index =
+            IndexController.LogicIndicesRegister.getPhysicalTableName(ProfileTaskRecord.INDEX_NAME);
+        final BoolQueryBuilder query = Query.bool();
 
         if (StringUtil.isNotEmpty(serviceId)) {
-            boolQueryBuilder.must().add(QueryBuilders.termQuery(ProfileTaskRecord.SERVICE_ID, serviceId));
+            query.must(Query.term(ProfileTaskRecord.SERVICE_ID, serviceId));
         }
 
         if (StringUtil.isNotEmpty(endpointName)) {
-            boolQueryBuilder.must().add(QueryBuilders.termQuery(ProfileTaskRecord.ENDPOINT_NAME, endpointName));
+            query.must(Query.term(ProfileTaskRecord.ENDPOINT_NAME, endpointName));
         }
 
         if (startTimeBucket != null) {
-            boolQueryBuilder.must()
-                            .add(QueryBuilders.rangeQuery(ProfileTaskRecord.TIME_BUCKET).gte(startTimeBucket));
+            query.must(Query.range(ProfileTaskRecord.TIME_BUCKET).gte(startTimeBucket));
         }
 
         if (endTimeBucket != null) {
-            boolQueryBuilder.must().add(QueryBuilders.rangeQuery(ProfileTaskRecord.TIME_BUCKET).lte(endTimeBucket));
+            query.must(Query.range(ProfileTaskRecord.TIME_BUCKET).lte(endTimeBucket));
         }
 
+        final SearchBuilder search = Search.builder().query(query);
+
         if (limit != null) {
-            sourceBuilder.size(limit);
+            search.size(limit);
         } else {
-            sourceBuilder.size(queryMaxSize);
+            search.size(queryMaxSize);
         }
 
-        sourceBuilder.sort(ProfileTaskRecord.START_TIME, SortOrder.DESC);
+        search.sort(ProfileTaskRecord.START_TIME, Sort.Order.DESC);
 
-        final SearchResponse response = getClient().search(
-            IndexController.LogicIndicesRegister.getPhysicalTableName(ProfileTaskRecord.INDEX_NAME), sourceBuilder);
+        final SearchResponse response = getClient().search(index, search.build());
 
         final LinkedList<ProfileTask> tasks = new LinkedList<>();
         for (SearchHit searchHit : response.getHits().getHits()) {
@@ -93,37 +95,40 @@ public class ProfileTaskQueryEsDAO extends EsDAO implements IProfileTaskQueryDAO
         if (StringUtil.isEmpty(id)) {
             return null;
         }
+        final String index =
+            IndexController.LogicIndicesRegister.getPhysicalTableName(ProfileTaskRecord.INDEX_NAME);
 
-        SearchSourceBuilder sourceBuilder = SearchSourceBuilder.searchSource();
-        sourceBuilder.query(QueryBuilders.idsQuery().addIds(id));
-        sourceBuilder.size(1);
+        final SearchBuilder search = Search.builder()
+                                           .query(Query.ids(id))
+                                           .size(1);
 
-        final SearchResponse response = getClient().search(
-            IndexController.LogicIndicesRegister.getPhysicalTableName(ProfileTaskRecord.INDEX_NAME), sourceBuilder);
+        final SearchResponse response = getClient().search(index, search.build());
 
-        if (response.getHits().getHits().length > 0) {
-            return parseTask(response.getHits().getHits()[0]);
+        if (response.getHits().getHits().size() > 0) {
+            return parseTask(response.getHits().getHits().iterator().next());
         }
 
         return null;
     }
 
     private ProfileTask parseTask(SearchHit data) {
+        final Map<String, Object> source = data.getSource();
         return ProfileTask.builder()
                           .id(data.getId())
-                          .serviceId((String) data.getSourceAsMap().get(ProfileTaskRecord.SERVICE_ID))
-                          .endpointName((String) data.getSourceAsMap().get(ProfileTaskRecord.ENDPOINT_NAME))
-                          .startTime(((Number) data.getSourceAsMap().get(ProfileTaskRecord.START_TIME)).longValue())
-                          .createTime(((Number) data.getSourceAsMap()
-                                                    .get(ProfileTaskRecord.CREATE_TIME)).longValue())
-                          .duration(((Number) data.getSourceAsMap().get(ProfileTaskRecord.DURATION)).intValue())
-                          .minDurationThreshold(((Number) data.getSourceAsMap()
-                                                              .get(
-                                                                  ProfileTaskRecord.MIN_DURATION_THRESHOLD)).intValue())
-                          .dumpPeriod(((Number) data.getSourceAsMap()
-                                                    .get(ProfileTaskRecord.DUMP_PERIOD)).intValue())
-                          .maxSamplingCount(((Number) data.getSourceAsMap()
-                                                          .get(ProfileTaskRecord.MAX_SAMPLING_COUNT)).intValue())
+                          .serviceId((String) source.get(ProfileTaskRecord.SERVICE_ID))
+                          .endpointName((String) source.get(ProfileTaskRecord.ENDPOINT_NAME))
+                          .startTime(
+                              ((Number) source.get(ProfileTaskRecord.START_TIME)).longValue())
+                          .createTime(
+                              ((Number) source.get(ProfileTaskRecord.CREATE_TIME)).longValue())
+                          .duration(((Number) source.get(ProfileTaskRecord.DURATION)).intValue())
+                          .minDurationThreshold(((Number) source.get(
+                              ProfileTaskRecord.MIN_DURATION_THRESHOLD)).intValue())
+                          .dumpPeriod(
+                              ((Number) source.get(ProfileTaskRecord.DUMP_PERIOD)).intValue())
+                          .maxSamplingCount(
+                              ((Number) source.get(ProfileTaskRecord.MAX_SAMPLING_COUNT))
+                                  .intValue())
                           .build();
     }
 }
diff --git a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/ProfileThreadSnapshotQueryEsDAO.java b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/ProfileThreadSnapshotQueryEsDAO.java
index 08921fe..e1bb0b2 100644
--- a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/ProfileThreadSnapshotQueryEsDAO.java
+++ b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/ProfileThreadSnapshotQueryEsDAO.java
@@ -25,6 +25,16 @@ import java.util.Base64;
 import java.util.Collections;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map;
+import org.apache.skywalking.library.elasticsearch.requests.search.BoolQueryBuilder;
+import org.apache.skywalking.library.elasticsearch.requests.search.Query;
+import org.apache.skywalking.library.elasticsearch.requests.search.Search;
+import org.apache.skywalking.library.elasticsearch.requests.search.SearchBuilder;
+import org.apache.skywalking.library.elasticsearch.requests.search.Sort;
+import org.apache.skywalking.library.elasticsearch.requests.search.aggregation.Aggregation;
+import org.apache.skywalking.library.elasticsearch.requests.search.aggregation.AggregationBuilder;
+import org.apache.skywalking.library.elasticsearch.response.search.SearchHit;
+import org.apache.skywalking.library.elasticsearch.response.search.SearchResponse;
 import org.apache.skywalking.oap.server.core.analysis.IDManager;
 import org.apache.skywalking.oap.server.core.analysis.manual.segment.SegmentRecord;
 import org.apache.skywalking.oap.server.core.profile.ProfileThreadSnapshotRecord;
@@ -35,16 +45,6 @@ import org.apache.skywalking.oap.server.library.util.BooleanUtils;
 import org.apache.skywalking.oap.server.library.util.CollectionUtils;
 import org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base.EsDAO;
 import org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base.IndexController;
-import org.elasticsearch.action.search.SearchResponse;
-import org.elasticsearch.index.query.BoolQueryBuilder;
-import org.elasticsearch.index.query.QueryBuilder;
-import org.elasticsearch.index.query.QueryBuilders;
-import org.elasticsearch.search.SearchHit;
-import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
-import org.elasticsearch.search.aggregations.AggregationBuilders;
-import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregation;
-import org.elasticsearch.search.builder.SearchSourceBuilder;
-import org.elasticsearch.search.sort.SortOrder;
 
 public class ProfileThreadSnapshotQueryEsDAO extends EsDAO implements IProfileThreadSnapshotQueryDAO {
 
@@ -59,62 +59,59 @@ public class ProfileThreadSnapshotQueryEsDAO extends EsDAO implements IProfileTh
 
     @Override
     public List<BasicTrace> queryProfiledSegments(String taskId) throws IOException {
-        // search segment id list
-        SearchSourceBuilder sourceBuilder = SearchSourceBuilder.searchSource();
-
-        BoolQueryBuilder boolQueryBuilder = QueryBuilders.boolQuery();
-        sourceBuilder.query(boolQueryBuilder);
-
-        boolQueryBuilder.must().add(QueryBuilders.termQuery(ProfileThreadSnapshotRecord.TASK_ID, taskId));
-        boolQueryBuilder.must().add(QueryBuilders.termQuery(ProfileThreadSnapshotRecord.SEQUENCE, 0));
-
-        sourceBuilder.size(querySegmentMaxSize);
-        sourceBuilder.sort(ProfileThreadSnapshotRecord.DUMP_TIME, SortOrder.DESC);
-
-        SearchResponse response = getClient().search(
+        final BoolQueryBuilder segmentIdQuery =
+            Query.bool()
+                 .must(Query.term(ProfileThreadSnapshotRecord.TASK_ID, taskId))
+                 .must(Query.term(ProfileThreadSnapshotRecord.SEQUENCE, 0));
+
+        final SearchBuilder search =
+            Search.builder().query(segmentIdQuery)
+                  .size(querySegmentMaxSize)
+                  .sort(
+                      ProfileThreadSnapshotRecord.DUMP_TIME,
+                      Sort.Order.DESC
+                  );
+
+        org.apache.skywalking.library.elasticsearch.response.search.SearchResponse response = getClient().search(
             IndexController.LogicIndicesRegister.getPhysicalTableName(ProfileThreadSnapshotRecord.INDEX_NAME),
-            sourceBuilder
+            search.build()
         );
 
-        final LinkedList<String> segments = new LinkedList<>();
-        for (SearchHit searchHit : response.getHits().getHits()) {
-            segments.add((String) searchHit.getSourceAsMap().get(ProfileThreadSnapshotRecord.SEGMENT_ID));
+        final List<String> segmentIds = new LinkedList<>();
+        for (org.apache.skywalking.library.elasticsearch.response.search.SearchHit searchHit : response.getHits().getHits()) {
+            segmentIds.add((String) searchHit.getSource().get(ProfileThreadSnapshotRecord.SEGMENT_ID));
         }
 
-        if (CollectionUtils.isEmpty(segments)) {
+        if (CollectionUtils.isEmpty(segmentIds)) {
             return Collections.emptyList();
         }
 
-        // search traces
-        sourceBuilder = SearchSourceBuilder.searchSource();
-
-        boolQueryBuilder = QueryBuilders.boolQuery();
-        sourceBuilder.query(boolQueryBuilder);
-        List<QueryBuilder> shouldQueryList = boolQueryBuilder.should();
-
-        for (String segmentId : segments) {
-            shouldQueryList.add(QueryBuilders.termQuery(SegmentRecord.SEGMENT_ID, segmentId));
+        final BoolQueryBuilder traceQuery = Query.bool();
+        for (String segmentId : segmentIds) {
+            traceQuery.should(Query.term(SegmentRecord.SEGMENT_ID, segmentId));
         }
-        sourceBuilder.size(segments.size());
-        sourceBuilder.sort(SegmentRecord.START_TIME, SortOrder.DESC);
+        final SearchBuilder traceSearch =
+            Search.builder().query(traceQuery)
+                  .size(segmentIds.size())
+                  .sort(SegmentRecord.START_TIME, Sort.Order.DESC);
 
-        response = getClient().search(SegmentRecord.INDEX_NAME, sourceBuilder);
+        response = getClient().search(SegmentRecord.INDEX_NAME, traceSearch.build());
 
         List<BasicTrace> result = new ArrayList<>();
         for (SearchHit searchHit : response.getHits().getHits()) {
             BasicTrace basicTrace = new BasicTrace();
 
-            basicTrace.setSegmentId((String) searchHit.getSourceAsMap().get(SegmentRecord.SEGMENT_ID));
-            basicTrace.setStart(String.valueOf(searchHit.getSourceAsMap().get(SegmentRecord.START_TIME)));
+            basicTrace.setSegmentId((String) searchHit.getSource().get(SegmentRecord.SEGMENT_ID));
+            basicTrace.setStart(String.valueOf(searchHit.getSource().get(SegmentRecord.START_TIME)));
             basicTrace.getEndpointNames().add(
                 IDManager.EndpointID.analysisId(
-                    (String) searchHit.getSourceAsMap().get(SegmentRecord.ENDPOINT_ID)
+                    (String) searchHit.getSource().get(SegmentRecord.ENDPOINT_ID)
                 ).getEndpointName());
-            basicTrace.setDuration(((Number) searchHit.getSourceAsMap().get(SegmentRecord.LATENCY)).intValue());
-            basicTrace.setError(BooleanUtils.valueToBoolean(((Number) searchHit.getSourceAsMap()
+            basicTrace.setDuration(((Number) searchHit.getSource().get(SegmentRecord.LATENCY)).intValue());
+            basicTrace.setError(BooleanUtils.valueToBoolean(((Number) searchHit.getSource()
                                                                                .get(
                                                                                    SegmentRecord.IS_ERROR)).intValue()));
-            basicTrace.getTraceIds().add((String) searchHit.getSourceAsMap().get(SegmentRecord.TRACE_ID));
+            basicTrace.getTraceIds().add((String) searchHit.getSource().get(SegmentRecord.TRACE_ID));
 
             result.add(basicTrace);
         }
@@ -125,7 +122,7 @@ public class ProfileThreadSnapshotQueryEsDAO extends EsDAO implements IProfileTh
     @Override
     public int queryMinSequence(String segmentId, long start, long end) throws IOException {
         return querySequenceWithAgg(
-            AggregationBuilders.min(ProfileThreadSnapshotRecord.SEQUENCE).field(ProfileThreadSnapshotRecord.SEQUENCE),
+            Aggregation.min(ProfileThreadSnapshotRecord.SEQUENCE).field(ProfileThreadSnapshotRecord.SEQUENCE),
             segmentId, start, end
         );
     }
@@ -133,7 +130,7 @@ public class ProfileThreadSnapshotQueryEsDAO extends EsDAO implements IProfileTh
     @Override
     public int queryMaxSequence(String segmentId, long start, long end) throws IOException {
         return querySequenceWithAgg(
-            AggregationBuilders.max(ProfileThreadSnapshotRecord.SEQUENCE).field(ProfileThreadSnapshotRecord.SEQUENCE),
+            Aggregation.max(ProfileThreadSnapshotRecord.SEQUENCE).field(ProfileThreadSnapshotRecord.SEQUENCE),
             segmentId, start, end
         );
     }
@@ -142,26 +139,26 @@ public class ProfileThreadSnapshotQueryEsDAO extends EsDAO implements IProfileTh
     public List<ProfileThreadSnapshotRecord> queryRecords(String segmentId,
                                                           int minSequence,
                                                           int maxSequence) throws IOException {
-        // search traces
-        SearchSourceBuilder sourceBuilder = SearchSourceBuilder.searchSource();
+        final String index = IndexController.LogicIndicesRegister.getPhysicalTableName(
+            ProfileThreadSnapshotRecord.INDEX_NAME);
 
-        BoolQueryBuilder boolQueryBuilder = QueryBuilders.boolQuery();
-        sourceBuilder.query(boolQueryBuilder);
-        List<QueryBuilder> mustQueryList = boolQueryBuilder.must();
+        final org.apache.skywalking.library.elasticsearch.requests.search.BoolQueryBuilder query =
+            Query.bool();
 
-        mustQueryList.add(QueryBuilders.termQuery(ProfileThreadSnapshotRecord.SEGMENT_ID, segmentId));
-        mustQueryList.add(
-            QueryBuilders.rangeQuery(ProfileThreadSnapshotRecord.SEQUENCE).gte(minSequence).lt(maxSequence));
-        sourceBuilder.size(maxSequence - minSequence);
+        query.must(Query.term(ProfileThreadSnapshotRecord.SEGMENT_ID, segmentId))
+             .must(Query.range(ProfileThreadSnapshotRecord.SEQUENCE)
+                        .gte(minSequence)
+                        .lt(maxSequence));
 
-        SearchResponse response = getClient().search(
-            IndexController.LogicIndicesRegister.getPhysicalTableName(ProfileThreadSnapshotRecord.INDEX_NAME),
-            sourceBuilder
-        );
+        final SearchBuilder search =
+            Search.builder().query(query)
+                  .size(maxSequence - minSequence);
+
+        org.apache.skywalking.library.elasticsearch.response.search.SearchResponse response = getClient().search(index, search.build());
 
         List<ProfileThreadSnapshotRecord> result = new ArrayList<>(maxSequence - minSequence);
-        for (SearchHit searchHit : response.getHits().getHits()) {
-            ProfileThreadSnapshotRecord record = builder.storage2Entity(searchHit.getSourceAsMap());
+        for (org.apache.skywalking.library.elasticsearch.response.search.SearchHit searchHit : response.getHits().getHits()) {
+            ProfileThreadSnapshotRecord record = builder.storage2Entity(searchHit.getSource());
 
             result.add(record);
         }
@@ -170,53 +167,55 @@ public class ProfileThreadSnapshotQueryEsDAO extends EsDAO implements IProfileTh
 
     @Override
     public SegmentRecord getProfiledSegment(String segmentId) throws IOException {
-        SearchSourceBuilder sourceBuilder = SearchSourceBuilder.searchSource();
-        sourceBuilder.query(QueryBuilders.termQuery(SegmentRecord.SEGMENT_ID, segmentId));
-        sourceBuilder.size(1);
+        final String index =
+            IndexController.LogicIndicesRegister.getPhysicalTableName(SegmentRecord.INDEX_NAME);
+        final SearchBuilder search =
+            Search.builder()
+                  .query(Query.term(SegmentRecord.SEGMENT_ID, segmentId))
+                  .size(1);
 
-        SearchResponse response = getClient().search(
-            IndexController.LogicIndicesRegister.getPhysicalTableName(SegmentRecord.INDEX_NAME), sourceBuilder);
+        org.apache.skywalking.library.elasticsearch.response.search.SearchResponse response = getClient().search(
+            index, search.build());
 
-        if (response.getHits().getHits().length == 0) {
+        if (response.getHits().getHits().isEmpty()) {
             return null;
         }
-        SearchHit searchHit = response.getHits().getHits()[0];
+        org.apache.skywalking.library.elasticsearch.response.search.SearchHit
+            searchHit = response.getHits().iterator().next();
         SegmentRecord segmentRecord = new SegmentRecord();
-        segmentRecord.setSegmentId((String) searchHit.getSourceAsMap().get(SegmentRecord.SEGMENT_ID));
-        segmentRecord.setTraceId((String) searchHit.getSourceAsMap().get(SegmentRecord.TRACE_ID));
-        segmentRecord.setServiceId((String) searchHit.getSourceAsMap().get(SegmentRecord.SERVICE_ID));
-        segmentRecord.setStartTime(((Number) searchHit.getSourceAsMap().get(SegmentRecord.START_TIME)).longValue());
-        segmentRecord.setLatency(((Number) searchHit.getSourceAsMap().get(SegmentRecord.LATENCY)).intValue());
-        segmentRecord.setIsError(((Number) searchHit.getSourceAsMap().get(SegmentRecord.IS_ERROR)).intValue());
-        String dataBinaryBase64 = (String) searchHit.getSourceAsMap().get(SegmentRecord.DATA_BINARY);
+        segmentRecord.setSegmentId((String) searchHit.getSource().get(SegmentRecord.SEGMENT_ID));
+        segmentRecord.setTraceId((String) searchHit.getSource().get(SegmentRecord.TRACE_ID));
+        segmentRecord.setServiceId((String) searchHit.getSource().get(SegmentRecord.SERVICE_ID));
+        segmentRecord.setStartTime(((Number) searchHit.getSource().get(SegmentRecord.START_TIME)).longValue());
+        segmentRecord.setLatency(((Number) searchHit.getSource().get(SegmentRecord.LATENCY)).intValue());
+        segmentRecord.setIsError(((Number) searchHit.getSource().get(SegmentRecord.IS_ERROR)).intValue());
+        String dataBinaryBase64 = (String) searchHit.getSource().get(SegmentRecord.DATA_BINARY);
         if (!Strings.isNullOrEmpty(dataBinaryBase64)) {
             segmentRecord.setDataBinary(Base64.getDecoder().decode(dataBinaryBase64));
         }
         return segmentRecord;
     }
 
-    protected int querySequenceWithAgg(AbstractAggregationBuilder aggregationBuilder,
+    protected int querySequenceWithAgg(AggregationBuilder aggregationBuilder,
                                        String segmentId,
                                        long start,
                                        long end) throws IOException {
-        SearchSourceBuilder sourceBuilder = SearchSourceBuilder.searchSource();
+        final org.apache.skywalking.library.elasticsearch.requests.search.BoolQueryBuilder query =
+            Query.bool();
 
-        BoolQueryBuilder boolQueryBuilder = QueryBuilders.boolQuery();
-        sourceBuilder.query(boolQueryBuilder);
-        List<QueryBuilder> mustQueryList = boolQueryBuilder.must();
+        query.must(Query.term(ProfileThreadSnapshotRecord.SEGMENT_ID, segmentId))
+             .must(Query.range(ProfileThreadSnapshotRecord.DUMP_TIME).gte(start).lte(end));
 
-        mustQueryList.add(QueryBuilders.termQuery(ProfileThreadSnapshotRecord.SEGMENT_ID, segmentId));
-        mustQueryList.add(QueryBuilders.rangeQuery(ProfileThreadSnapshotRecord.DUMP_TIME).gte(start).lte(end));
-        sourceBuilder.size(0);
+        final SearchBuilder search = Search.builder().query(query).size(0);
 
-        sourceBuilder.aggregation(aggregationBuilder);
-        SearchResponse response = getClient().search(
-            IndexController.LogicIndicesRegister.getPhysicalTableName(ProfileThreadSnapshotRecord.INDEX_NAME),
-            sourceBuilder
-        );
-        NumericMetricsAggregation.SingleValue agg = response.getAggregations()
-                                                            .get(ProfileThreadSnapshotRecord.SEQUENCE);
+        search.aggregation(aggregationBuilder);
+        final String index = IndexController.LogicIndicesRegister.getPhysicalTableName(
+            ProfileThreadSnapshotRecord.INDEX_NAME);
+        final SearchResponse response = getClient().search(index, search.build());
+        final Map<String, Object> agg =
+            (Map<String, Object>) response.getAggregations()
+                                          .get(ProfileThreadSnapshotRecord.SEQUENCE);
 
-        return (int) agg.value();
+        return ((Number) agg.get("value")).intValue();
     }
 }
diff --git a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/TopNRecordsQueryEsDAO.java b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/TopNRecordsQueryEsDAO.java
index cd72d6e..4e30f17 100644
--- a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/TopNRecordsQueryEsDAO.java
+++ b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/TopNRecordsQueryEsDAO.java
@@ -23,6 +23,13 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import org.apache.skywalking.apm.util.StringUtil;
+import org.apache.skywalking.library.elasticsearch.requests.search.BoolQueryBuilder;
+import org.apache.skywalking.library.elasticsearch.requests.search.Query;
+import org.apache.skywalking.library.elasticsearch.requests.search.Search;
+import org.apache.skywalking.library.elasticsearch.requests.search.SearchBuilder;
+import org.apache.skywalking.library.elasticsearch.requests.search.Sort;
+import org.apache.skywalking.library.elasticsearch.response.search.SearchHit;
+import org.apache.skywalking.library.elasticsearch.response.search.SearchResponse;
 import org.apache.skywalking.oap.server.core.analysis.IDManager;
 import org.apache.skywalking.oap.server.core.analysis.topn.TopN;
 import org.apache.skywalking.oap.server.core.query.enumeration.Order;
@@ -33,12 +40,6 @@ import org.apache.skywalking.oap.server.core.storage.query.ITopNRecordsQueryDAO;
 import org.apache.skywalking.oap.server.library.client.elasticsearch.ElasticSearchClient;
 import org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base.EsDAO;
 import org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base.IndexController;
-import org.elasticsearch.action.search.SearchResponse;
-import org.elasticsearch.index.query.BoolQueryBuilder;
-import org.elasticsearch.index.query.QueryBuilders;
-import org.elasticsearch.search.SearchHit;
-import org.elasticsearch.search.builder.SearchSourceBuilder;
-import org.elasticsearch.search.sort.SortOrder;
 
 public class TopNRecordsQueryEsDAO extends EsDAO implements ITopNRecordsQueryDAO {
     public TopNRecordsQueryEsDAO(ElasticSearchClient client) {
@@ -49,32 +50,41 @@ public class TopNRecordsQueryEsDAO extends EsDAO implements ITopNRecordsQueryDAO
     public List<SelectedRecord> readSampledRecords(final TopNCondition condition,
                                                    final String valueColumnName,
                                                    final Duration duration) throws IOException {
-        SearchSourceBuilder sourceBuilder = SearchSourceBuilder.searchSource();
-        BoolQueryBuilder boolQueryBuilder = QueryBuilders.boolQuery();
-        boolQueryBuilder.must().add(QueryBuilders.rangeQuery(TopN.TIME_BUCKET)
-                                                 .gte(duration.getStartTimeBucketInSec())
-                                                 .lte(duration.getEndTimeBucketInSec()));
+        final BoolQueryBuilder query =
+            Query.bool()
+                 .must(Query.range(TopN.TIME_BUCKET)
+                            .gte(duration.getStartTimeBucketInSec())
+                            .lte(duration.getEndTimeBucketInSec()));
 
         if (StringUtil.isNotEmpty(condition.getParentService())) {
-            final String serviceId = IDManager.ServiceID.buildId(condition.getParentService(), condition.isNormal());
-            boolQueryBuilder.must().add(QueryBuilders.termQuery(TopN.SERVICE_ID, serviceId));
+            final String serviceId =
+                IDManager.ServiceID.buildId(condition.getParentService(), condition.isNormal());
+            query.must(Query.term(TopN.SERVICE_ID, serviceId));
         }
 
-        sourceBuilder.query(boolQueryBuilder);
-        sourceBuilder.size(condition.getTopN())
-                     .sort(valueColumnName, condition.getOrder().equals(Order.DES) ? SortOrder.DESC : SortOrder.ASC);
-        SearchResponse response = getClient().search(
-            IndexController.LogicIndicesRegister.getPhysicalTableName(condition.getName()), sourceBuilder);
+        final SearchBuilder search =
+            Search.builder()
+                  .query(query)
+                  .size(condition.getTopN())
+                  .sort(
+                      valueColumnName,
+                      condition.getOrder().equals(Order.DES) ?
+                          Sort.Order.DESC : Sort.Order.ASC
+                  );
+        final SearchResponse response = getClient().search(
+            IndexController.LogicIndicesRegister.getPhysicalTableName(condition.getName()),
+            search.build()
+        );
 
         List<SelectedRecord> results = new ArrayList<>(condition.getTopN());
 
         for (SearchHit searchHit : response.getHits().getHits()) {
             SelectedRecord record = new SelectedRecord();
-            final Map<String, Object> sourceAsMap = searchHit.getSourceAsMap();
+            final Map<String, Object> sourceAsMap = searchHit.getSource();
             record.setName((String) sourceAsMap.get(TopN.STATEMENT));
             record.setRefId((String) sourceAsMap.get(TopN.TRACE_ID));
             record.setId(record.getRefId());
-            record.setValue(((Number) sourceAsMap.get(valueColumnName)).toString());
+            record.setValue(sourceAsMap.get(valueColumnName).toString());
             results.add(record);
         }
 
diff --git a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/TopologyQueryEsDAO.java b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/TopologyQueryEsDAO.java
index b18317b..3f399a7 100644
--- a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/TopologyQueryEsDAO.java
+++ b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/TopologyQueryEsDAO.java
@@ -21,6 +21,13 @@ package org.apache.skywalking.oap.server.storage.plugin.elasticsearch.query;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
+import org.apache.skywalking.library.elasticsearch.requests.search.BoolQueryBuilder;
+import org.apache.skywalking.library.elasticsearch.requests.search.Query;
+import org.apache.skywalking.library.elasticsearch.requests.search.Search;
+import org.apache.skywalking.library.elasticsearch.requests.search.SearchBuilder;
+import org.apache.skywalking.library.elasticsearch.requests.search.aggregation.Aggregation;
+import org.apache.skywalking.library.elasticsearch.response.search.SearchResponse;
 import org.apache.skywalking.oap.server.core.UnexpectedException;
 import org.apache.skywalking.oap.server.core.analysis.manual.relation.endpoint.EndpointRelationServerSideMetrics;
 import org.apache.skywalking.oap.server.core.analysis.manual.relation.instance.ServiceInstanceRelationClientSideMetrics;
@@ -35,12 +42,6 @@ import org.apache.skywalking.oap.server.library.client.elasticsearch.ElasticSear
 import org.apache.skywalking.oap.server.library.util.CollectionUtils;
 import org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base.EsDAO;
 import org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base.IndexController;
-import org.elasticsearch.action.search.SearchResponse;
-import org.elasticsearch.index.query.BoolQueryBuilder;
-import org.elasticsearch.index.query.QueryBuilders;
-import org.elasticsearch.search.aggregations.AggregationBuilders;
-import org.elasticsearch.search.aggregations.bucket.terms.Terms;
-import org.elasticsearch.search.builder.SearchSourceBuilder;
 
 public class TopologyQueryEsDAO extends EsDAO implements ITopologyQueryDAO {
 
@@ -49,65 +50,70 @@ public class TopologyQueryEsDAO extends EsDAO implements ITopologyQueryDAO {
     }
 
     @Override
-    public List<Call.CallDetail> loadServiceRelationsDetectedAtServerSide(long startTB,
-                                                                          long endTB,
-                                                                          List<String> serviceIds) throws IOException {
+    public List<Call.CallDetail> loadServiceRelationsDetectedAtServerSide(
+        long startTB, long endTB, List<String> serviceIds) throws IOException {
         if (CollectionUtils.isEmpty(serviceIds)) {
             throw new UnexpectedException("Service id is empty");
         }
 
-        SearchSourceBuilder sourceBuilder = SearchSourceBuilder.searchSource();
-        sourceBuilder.size(0);
+        final SearchBuilder sourceBuilder = Search.builder().size(0);
         setQueryCondition(sourceBuilder, startTB, endTB, serviceIds);
 
-        return buildServiceRelation(sourceBuilder, ServiceRelationServerSideMetrics.INDEX_NAME, DetectPoint.SERVER);
+        return buildServiceRelation(
+            sourceBuilder, ServiceRelationServerSideMetrics.INDEX_NAME, DetectPoint.SERVER);
     }
 
     @Override
     public List<Call.CallDetail> loadServiceRelationDetectedAtClientSide(long startTB,
                                                                          long endTB,
-                                                                         List<String> serviceIds) throws IOException {
+                                                                         List<String> serviceIds)
+        throws IOException {
         if (CollectionUtils.isEmpty(serviceIds)) {
             throw new UnexpectedException("Service id is empty");
         }
 
-        SearchSourceBuilder sourceBuilder = SearchSourceBuilder.searchSource();
-        sourceBuilder.size(0);
+        final SearchBuilder sourceBuilder = Search.builder().size(0);
         setQueryCondition(sourceBuilder, startTB, endTB, serviceIds);
 
-        return buildServiceRelation(sourceBuilder, ServiceRelationClientSideMetrics.INDEX_NAME, DetectPoint.CLIENT);
+        return buildServiceRelation(
+            sourceBuilder, ServiceRelationClientSideMetrics.INDEX_NAME, DetectPoint.CLIENT);
     }
 
     @Override
     public List<Call.CallDetail> loadServiceRelationsDetectedAtServerSide(long startTB,
-                                                                          long endTB) throws IOException {
-        SearchSourceBuilder sourceBuilder = SearchSourceBuilder.searchSource();
-        sourceBuilder.query(QueryBuilders.rangeQuery(ServiceRelationServerSideMetrics.TIME_BUCKET)
-                                         .gte(startTB)
-                                         .lte(endTB));
-        sourceBuilder.size(0);
-
-        return buildServiceRelation(sourceBuilder, ServiceRelationServerSideMetrics.INDEX_NAME, DetectPoint.SERVER);
+                                                                          long endTB)
+        throws IOException {
+        SearchBuilder sourceBuilder = Search.builder();
+        sourceBuilder.query(Query.range(ServiceRelationServerSideMetrics.TIME_BUCKET)
+                                 .gte(startTB)
+                                 .lte(endTB))
+                     .size(0);
+
+        return buildServiceRelation(
+            sourceBuilder, ServiceRelationServerSideMetrics.INDEX_NAME, DetectPoint.SERVER);
     }
 
     @Override
     public List<Call.CallDetail> loadServiceRelationDetectedAtClientSide(long startTB,
-                                                                         long endTB) throws IOException {
-        SearchSourceBuilder sourceBuilder = SearchSourceBuilder.searchSource();
-        sourceBuilder.query(QueryBuilders.rangeQuery(ServiceRelationServerSideMetrics.TIME_BUCKET)
-                                         .gte(startTB)
-                                         .lte(endTB));
-        sourceBuilder.size(0);
-
-        return buildServiceRelation(sourceBuilder, ServiceRelationClientSideMetrics.INDEX_NAME, DetectPoint.CLIENT);
+                                                                         long endTB)
+        throws IOException {
+        SearchBuilder sourceBuilder = Search.builder();
+        sourceBuilder.query(Query.range(ServiceRelationServerSideMetrics.TIME_BUCKET)
+                                 .gte(startTB)
+                                 .lte(endTB))
+                     .size(0);
+
+        return buildServiceRelation(
+            sourceBuilder, ServiceRelationClientSideMetrics.INDEX_NAME, DetectPoint.CLIENT);
     }
 
     @Override
     public List<Call.CallDetail> loadInstanceRelationDetectedAtServerSide(String clientServiceId,
                                                                           String serverServiceId,
                                                                           long startTB,
-                                                                          long endTB) throws IOException {
-        SearchSourceBuilder sourceBuilder = SearchSourceBuilder.searchSource();
+                                                                          long endTB)
+        throws IOException {
+        SearchBuilder sourceBuilder = Search.builder();
         sourceBuilder.size(0);
         setInstanceQueryCondition(sourceBuilder, startTB, endTB, clientServiceId, serverServiceId);
 
@@ -119,8 +125,9 @@ public class TopologyQueryEsDAO extends EsDAO implements ITopologyQueryDAO {
     public List<Call.CallDetail> loadInstanceRelationDetectedAtClientSide(String clientServiceId,
                                                                           String serverServiceId,
                                                                           long startTB,
-                                                                          long endTB) throws IOException {
-        SearchSourceBuilder sourceBuilder = SearchSourceBuilder.searchSource();
+                                                                          long endTB)
+        throws IOException {
+        SearchBuilder sourceBuilder = Search.builder();
         sourceBuilder.size(0);
         setInstanceQueryCondition(sourceBuilder, startTB, endTB, clientServiceId, serverServiceId);
 
@@ -128,30 +135,35 @@ public class TopologyQueryEsDAO extends EsDAO implements ITopologyQueryDAO {
             sourceBuilder, ServiceInstanceRelationClientSideMetrics.INDEX_NAME, DetectPoint.CLIENT);
     }
 
-    private void setInstanceQueryCondition(SearchSourceBuilder sourceBuilder, long startTB, long endTB,
+    private void setInstanceQueryCondition(SearchBuilder sourceBuilder, long startTB,
+                                           long endTB,
                                            String clientServiceId, String serverServiceId) {
-        BoolQueryBuilder boolQuery = QueryBuilders.boolQuery();
-        boolQuery.must()
-                 .add(QueryBuilders.rangeQuery(EndpointRelationServerSideMetrics.TIME_BUCKET).gte(startTB).lte(endTB));
+        BoolQueryBuilder boolQuery = Query.bool();
+        boolQuery.must(Query.range(EndpointRelationServerSideMetrics.TIME_BUCKET)
+                            .gte(startTB).lte(endTB));
 
-        BoolQueryBuilder serviceIdBoolQuery = new BoolQueryBuilder();
+        BoolQueryBuilder serviceIdBoolQuery = Query.bool();
         boolQuery.must(serviceIdBoolQuery);
 
-        BoolQueryBuilder serverRelationBoolQuery = new BoolQueryBuilder();
+        BoolQueryBuilder serverRelationBoolQuery = Query.bool();
         serviceIdBoolQuery.should(serverRelationBoolQuery);
 
         serverRelationBoolQuery.must(
-            QueryBuilders.termQuery(ServiceInstanceRelationServerSideMetrics.SOURCE_SERVICE_ID, clientServiceId));
+            Query.term(
+                ServiceInstanceRelationServerSideMetrics.SOURCE_SERVICE_ID, clientServiceId));
         serverRelationBoolQuery.must(
-            QueryBuilders.termQuery(ServiceInstanceRelationServerSideMetrics.DEST_SERVICE_ID, serverServiceId));
+            Query.term(
+                ServiceInstanceRelationServerSideMetrics.DEST_SERVICE_ID, serverServiceId));
 
-        BoolQueryBuilder clientRelationBoolQuery = new BoolQueryBuilder();
+        BoolQueryBuilder clientRelationBoolQuery = Query.bool();
         serviceIdBoolQuery.should(clientRelationBoolQuery);
 
         clientRelationBoolQuery.must(
-            QueryBuilders.termQuery(ServiceInstanceRelationServerSideMetrics.DEST_SERVICE_ID, clientServiceId));
+            Query.term(
+                ServiceInstanceRelationServerSideMetrics.DEST_SERVICE_ID, clientServiceId));
         clientRelationBoolQuery.must(
-            QueryBuilders.termQuery(ServiceInstanceRelationServerSideMetrics.SOURCE_SERVICE_ID, serverServiceId));
+            Query.term(
+                ServiceInstanceRelationServerSideMetrics.SOURCE_SERVICE_ID, serverServiceId));
 
         sourceBuilder.query(boolQuery);
     }
@@ -160,48 +172,60 @@ public class TopologyQueryEsDAO extends EsDAO implements ITopologyQueryDAO {
     public List<Call.CallDetail> loadEndpointRelation(long startTB,
                                                       long endTB,
                                                       String destEndpointId) throws IOException {
-        SearchSourceBuilder sourceBuilder = SearchSourceBuilder.searchSource();
+        SearchBuilder sourceBuilder = Search.builder();
         sourceBuilder.size(0);
 
-        BoolQueryBuilder boolQuery = QueryBuilders.boolQuery();
-        boolQuery.must()
-                 .add(QueryBuilders.rangeQuery(EndpointRelationServerSideMetrics.TIME_BUCKET).gte(startTB).lte(endTB));
-
-        BoolQueryBuilder serviceIdBoolQuery = QueryBuilders.boolQuery();
-        boolQuery.must().add(serviceIdBoolQuery);
-        serviceIdBoolQuery.should()
-                          .add(QueryBuilders.termQuery(
-                              EndpointRelationServerSideMetrics.SOURCE_ENDPOINT, destEndpointId
-                          ));
-        serviceIdBoolQuery.should()
-                          .add(QueryBuilders.termQuery(
-                              EndpointRelationServerSideMetrics.DEST_ENDPOINT, destEndpointId
-                          ));
+        BoolQueryBuilder boolQuery = Query.bool();
+        boolQuery.must(
+            Query.range(EndpointRelationServerSideMetrics.TIME_BUCKET)
+                 .gte(startTB).lte(endTB));
+
+        BoolQueryBuilder serviceIdBoolQuery = Query.bool();
+        boolQuery.must(serviceIdBoolQuery);
+        serviceIdBoolQuery.should(
+            Query.term(
+                EndpointRelationServerSideMetrics.SOURCE_ENDPOINT, destEndpointId
+            ));
+        serviceIdBoolQuery.must(
+            Query.term(
+                EndpointRelationServerSideMetrics.DEST_ENDPOINT, destEndpointId
+            ));
 
         sourceBuilder.query(boolQuery);
 
-        return loadEndpoint(sourceBuilder, EndpointRelationServerSideMetrics.INDEX_NAME, DetectPoint.SERVER);
+        return loadEndpoint(
+            sourceBuilder, EndpointRelationServerSideMetrics.INDEX_NAME, DetectPoint.SERVER);
     }
 
-    private List<Call.CallDetail> buildServiceRelation(SearchSourceBuilder sourceBuilder, String indexName,
+    private List<Call.CallDetail> buildServiceRelation(SearchBuilder sourceBuilder,
+                                                       String indexName,
                                                        DetectPoint detectPoint) throws IOException {
         sourceBuilder.aggregation(
-            AggregationBuilders
+            Aggregation
                 .terms(Metrics.ENTITY_ID).field(Metrics.ENTITY_ID)
                 .subAggregation(
-                    AggregationBuilders.terms(ServiceRelationServerSideMetrics.COMPONENT_ID)
-                                       .field(ServiceRelationServerSideMetrics.COMPONENT_ID))
+                    Aggregation.terms(ServiceRelationServerSideMetrics.COMPONENT_ID)
+                               .field(ServiceRelationServerSideMetrics.COMPONENT_ID))
                 .size(1000));
 
-        SearchResponse response = getClient().search(
-            IndexController.LogicIndicesRegister.getPhysicalTableName(indexName), sourceBuilder);
-
-        List<Call.CallDetail> calls = new ArrayList<>();
-        Terms entityTerms = response.getAggregations().get(Metrics.ENTITY_ID);
-        for (Terms.Bucket entityBucket : entityTerms.getBuckets()) {
-            String entityId = entityBucket.getKeyAsString();
-            Terms componentTerms = entityBucket.getAggregations().get(ServiceRelationServerSideMetrics.COMPONENT_ID);
-            final int componentId = componentTerms.getBuckets().get(0).getKeyAsNumber().intValue();
+        final String index =
+            IndexController.LogicIndicesRegister.getPhysicalTableName(indexName);
+        final SearchResponse response = getClient().search(index, sourceBuilder.build());
+
+        final List<Call.CallDetail> calls = new ArrayList<>();
+        final Map<String, Object> entityTerms =
+            (Map<String, Object>) response.getAggregations().get(Metrics.ENTITY_ID);
+        final List<Map<String, Object>> buckets =
+            (List<Map<String, Object>>) entityTerms.get("buckets");
+        for (final Map<String, Object> entityBucket : buckets) {
+            String entityId = (String) entityBucket.get("key");
+            final Map<String, Object> componentTerms =
+                (Map<String, Object>) entityBucket.get(
+                    ServiceRelationServerSideMetrics.COMPONENT_ID);
+            final Map<String, Object> subAgg =
+                (Map<String, Object>) componentTerms.get(
+                    ServiceRelationServerSideMetrics.COMPONENT_ID);
+            final int componentId = ((Number) subAgg.get("key")).intValue();
 
             Call.CallDetail call = new Call.CallDetail();
             call.buildFromServiceRelation(entityId, componentId, detectPoint);
@@ -210,26 +234,36 @@ public class TopologyQueryEsDAO extends EsDAO implements ITopologyQueryDAO {
         return calls;
     }
 
-    private List<Call.CallDetail> buildInstanceRelation(SearchSourceBuilder sourceBuilder, String indexName,
-                                                        DetectPoint detectPoint) throws IOException {
+    private List<Call.CallDetail> buildInstanceRelation(SearchBuilder sourceBuilder,
+                                                        String indexName,
+                                                        DetectPoint detectPoint)
+        throws IOException {
         sourceBuilder.aggregation(
-            AggregationBuilders
+            Aggregation
                 .terms(Metrics.ENTITY_ID).field(Metrics.ENTITY_ID)
                 .subAggregation(
-                    AggregationBuilders.terms(ServiceInstanceRelationServerSideMetrics.COMPONENT_ID)
-                                       .field(ServiceInstanceRelationServerSideMetrics.COMPONENT_ID))
+                    Aggregation.terms(ServiceInstanceRelationServerSideMetrics.COMPONENT_ID)
+                               .field(
+                                   ServiceInstanceRelationServerSideMetrics.COMPONENT_ID))
                 .size(1000));
 
-        SearchResponse response = getClient().search(
-            IndexController.LogicIndicesRegister.getPhysicalTableName(indexName), sourceBuilder);
+        final String index =
+            IndexController.LogicIndicesRegister.getPhysicalTableName(indexName);
+        SearchResponse response = getClient().search(index, sourceBuilder.build());
 
         List<Call.CallDetail> calls = new ArrayList<>();
-        Terms entityTerms = response.getAggregations().get(Metrics.ENTITY_ID);
-        for (Terms.Bucket entityBucket : entityTerms.getBuckets()) {
-            String entityId = entityBucket.getKeyAsString();
-            Terms componentTerms = entityBucket.getAggregations()
-                                               .get(ServiceInstanceRelationServerSideMetrics.COMPONENT_ID);
-            final int componentId = componentTerms.getBuckets().get(0).getKeyAsNumber().intValue();
+        final Map<String, Object> entityTerms =
+            (Map<String, Object>) response.getAggregations().get(Metrics.ENTITY_ID);
+        final List<Map<String, Object>> buckets =
+            (List<Map<String, Object>>) entityTerms.get("buckets");
+        for (Map<String, Object> entityBucket : buckets) {
+            final String entityId = (String) entityBucket.get("key");
+            final Map<String, Object> componentTerms = (Map<String, Object>) entityBucket.get(
+                ServiceInstanceRelationServerSideMetrics.COMPONENT_ID);
+            final Map<String, Object> subAgg =
+                (Map<String, Object>) componentTerms.get(
+                    ServiceRelationServerSideMetrics.COMPONENT_ID);
+            final int componentId = ((Number) subAgg.get("key")).intValue();
 
             Call.CallDetail call = new Call.CallDetail();
             call.buildFromInstanceRelation(entityId, componentId, detectPoint);
@@ -238,17 +272,22 @@ public class TopologyQueryEsDAO extends EsDAO implements ITopologyQueryDAO {
         return calls;
     }
 
-    private List<Call.CallDetail> loadEndpoint(SearchSourceBuilder sourceBuilder, String indexName,
+    private List<Call.CallDetail> loadEndpoint(SearchBuilder sourceBuilder, String indexName,
                                                DetectPoint detectPoint) throws IOException {
-        sourceBuilder.aggregation(AggregationBuilders.terms(Metrics.ENTITY_ID).field(Metrics.ENTITY_ID).size(1000));
+        sourceBuilder.aggregation(
+            Aggregation.terms(Metrics.ENTITY_ID).field(Metrics.ENTITY_ID).size(1000));
 
-        SearchResponse response = getClient().search(
-            IndexController.LogicIndicesRegister.getPhysicalTableName(indexName), sourceBuilder);
+        final String index =
+            IndexController.LogicIndicesRegister.getPhysicalTableName(indexName);
+        final SearchResponse response = getClient().search(index, sourceBuilder.build());
 
-        List<Call.CallDetail> calls = new ArrayList<>();
-        Terms entityTerms = response.getAggregations().get(Metrics.ENTITY_ID);
-        for (Terms.Bucket entityBucket : entityTerms.getBuckets()) {
-            String entityId = entityBucket.getKeyAsString();
+        final List<Call.CallDetail> calls = new ArrayList<>();
+        final Map<String, Object> entityTerms =
+            (Map<String, Object>) response.getAggregations().get(Metrics.ENTITY_ID);
+        final List<Map<String, Object>> buckets =
+            (List<Map<String, Object>>) entityTerms.get("buckets");
+        for (final Map<String, Object> entityBucket : buckets) {
+            String entityId = (String) entityBucket.get("key");
 
             Call.CallDetail call = new Call.CallDetail();
             call.buildFromEndpointRelation(entityId, detectPoint);
@@ -257,37 +296,43 @@ public class TopologyQueryEsDAO extends EsDAO implements ITopologyQueryDAO {
         return calls;
     }
 
-    private void setQueryCondition(SearchSourceBuilder sourceBuilder, long startTB, long endTB,
+    private void setQueryCondition(SearchBuilder searchBuilder, long startTB, long endTB,
                                    List<String> serviceIds) {
-        BoolQueryBuilder boolQuery = QueryBuilders.boolQuery();
-        boolQuery.must()
-                 .add(QueryBuilders.rangeQuery(ServiceRelationServerSideMetrics.TIME_BUCKET).gte(startTB).lte(endTB));
+        final org.apache.skywalking.library.elasticsearch.requests.search.BoolQueryBuilder query =
+            Query.bool();
+
+        query.must(Query.range(ServiceRelationServerSideMetrics.TIME_BUCKET)
+                        .gte(startTB)
+                        .lte(endTB));
 
-        BoolQueryBuilder serviceIdBoolQuery = QueryBuilders.boolQuery();
-        boolQuery.must().add(serviceIdBoolQuery);
+        org.apache.skywalking.library.elasticsearch.requests.search.BoolQueryBuilder
+            serviceIdBoolQuery = Query.bool();
+
+        query.must(serviceIdBoolQuery);
 
         if (serviceIds.size() == 1) {
-            serviceIdBoolQuery.should()
-                              .add(
-                                  QueryBuilders.termQuery(ServiceRelationServerSideMetrics.SOURCE_SERVICE_ID, serviceIds
-                                      .get(0)));
-            serviceIdBoolQuery.should()
-                              .add(QueryBuilders.termQuery(
-                                  ServiceRelationServerSideMetrics.DEST_SERVICE_ID,
-                                  serviceIds.get(0)
-                              ));
+            serviceIdBoolQuery.should(
+                Query.term(
+                    ServiceRelationServerSideMetrics.SOURCE_SERVICE_ID,
+                    serviceIds.get(0)
+                ));
+            serviceIdBoolQuery.should(
+                Query.term(
+                    ServiceRelationServerSideMetrics.DEST_SERVICE_ID,
+                    serviceIds.get(0)
+                ));
         } else {
-            serviceIdBoolQuery.should()
-                              .add(QueryBuilders.termsQuery(
-                                  ServiceRelationServerSideMetrics.SOURCE_SERVICE_ID,
-                                  serviceIds
-                              ));
-            serviceIdBoolQuery.should()
-                              .add(QueryBuilders.termsQuery(
-                                  ServiceRelationServerSideMetrics.DEST_SERVICE_ID,
-                                  serviceIds
-                              ));
+            serviceIdBoolQuery.should(
+                Query.terms(
+                    ServiceRelationServerSideMetrics.SOURCE_SERVICE_ID,
+                    serviceIds
+                ));
+            serviceIdBoolQuery.should(
+                Query.terms(
+                    ServiceRelationServerSideMetrics.DEST_SERVICE_ID,
+                    serviceIds
+                ));
         }
-        sourceBuilder.query(boolQuery);
+        searchBuilder.query(query);
     }
 }
diff --git a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/TraceQueryEsDAO.java b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/TraceQueryEsDAO.java
index 8468b54..c36730e 100644
--- a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/TraceQueryEsDAO.java
+++ b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/TraceQueryEsDAO.java
@@ -25,6 +25,12 @@ import java.util.Base64;
 import java.util.Collections;
 import java.util.List;
 import org.apache.skywalking.apm.util.StringUtil;
+import org.apache.skywalking.library.elasticsearch.requests.search.BoolQueryBuilder;
+import org.apache.skywalking.library.elasticsearch.requests.search.Query;
+import org.apache.skywalking.library.elasticsearch.requests.search.RangeQueryBuilder;
+import org.apache.skywalking.library.elasticsearch.requests.search.Search;
+import org.apache.skywalking.library.elasticsearch.requests.search.SearchBuilder;
+import org.apache.skywalking.library.elasticsearch.requests.search.Sort;
 import org.apache.skywalking.oap.server.core.analysis.IDManager;
 import org.apache.skywalking.oap.server.core.analysis.manual.searchtag.Tag;
 import org.apache.skywalking.oap.server.core.analysis.manual.segment.SegmentRecord;
@@ -41,17 +47,11 @@ import org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base.EsDAO;
 import org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base.IndexController;
 import org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base.TimeRangeIndexNameMaker;
 import org.elasticsearch.action.search.SearchResponse;
-import org.elasticsearch.index.query.BoolQueryBuilder;
-import org.elasticsearch.index.query.QueryBuilder;
-import org.elasticsearch.index.query.QueryBuilders;
-import org.elasticsearch.index.query.RangeQueryBuilder;
 import org.elasticsearch.search.SearchHit;
-import org.elasticsearch.search.builder.SearchSourceBuilder;
-import org.elasticsearch.search.sort.SortOrder;
 
 public class TraceQueryEsDAO extends EsDAO implements ITraceQueryDAO {
 
-    private int segmentQueryMaxSize;
+    private final int segmentQueryMaxSize;
 
     public TraceQueryEsDAO(ElasticSearchClient client, int segmentQueryMaxSize) {
         super(client);
@@ -72,66 +72,67 @@ public class TraceQueryEsDAO extends EsDAO implements ITraceQueryDAO {
                                        TraceState traceState,
                                        QueryOrder queryOrder,
                                        final List<Tag> tags) throws IOException {
-        SearchSourceBuilder sourceBuilder = SearchSourceBuilder.searchSource();
-
-        BoolQueryBuilder boolQueryBuilder = QueryBuilders.boolQuery();
-        sourceBuilder.query(boolQueryBuilder);
-        List<QueryBuilder> mustQueryList = boolQueryBuilder.must();
+        final org.apache.skywalking.library.elasticsearch.requests.search.BoolQueryBuilder query =
+            Query.bool();
 
         if (startSecondTB != 0 && endSecondTB != 0) {
-            mustQueryList.add(QueryBuilders.rangeQuery(SegmentRecord.TIME_BUCKET).gte(startSecondTB).lte(endSecondTB));
+            query.must(Query.range(SegmentRecord.TIME_BUCKET).gte(startSecondTB).lte(endSecondTB));
         }
 
         if (minDuration != 0 || maxDuration != 0) {
-            RangeQueryBuilder rangeQueryBuilder = QueryBuilders.rangeQuery(SegmentRecord.LATENCY);
+            RangeQueryBuilder rangeQueryBuilder = Query.range(SegmentRecord.LATENCY);
             if (minDuration != 0) {
                 rangeQueryBuilder.gte(minDuration);
             }
             if (maxDuration != 0) {
                 rangeQueryBuilder.lte(maxDuration);
             }
-            boolQueryBuilder.must().add(rangeQueryBuilder);
+            query.must(rangeQueryBuilder);
         }
         if (StringUtil.isNotEmpty(serviceId)) {
-            boolQueryBuilder.must().add(QueryBuilders.termQuery(SegmentRecord.SERVICE_ID, serviceId));
+            query.must(Query.term(SegmentRecord.SERVICE_ID, serviceId));
         }
         if (StringUtil.isNotEmpty(serviceInstanceId)) {
-            boolQueryBuilder.must().add(QueryBuilders.termQuery(SegmentRecord.SERVICE_INSTANCE_ID, serviceInstanceId));
+            query.must(Query.term(SegmentRecord.SERVICE_INSTANCE_ID, serviceInstanceId));
         }
         if (!Strings.isNullOrEmpty(endpointId)) {
-            boolQueryBuilder.must().add(QueryBuilders.termQuery(SegmentRecord.ENDPOINT_ID, endpointId));
+            query.must(Query.term(SegmentRecord.ENDPOINT_ID, endpointId));
         }
         if (!Strings.isNullOrEmpty(traceId)) {
-            boolQueryBuilder.must().add(QueryBuilders.termQuery(SegmentRecord.TRACE_ID, traceId));
+            query.must(Query.term(SegmentRecord.TRACE_ID, traceId));
         }
         switch (traceState) {
             case ERROR:
-                mustQueryList.add(QueryBuilders.matchQuery(SegmentRecord.IS_ERROR, BooleanUtils.TRUE));
+                query.must(Query.match(SegmentRecord.IS_ERROR, BooleanUtils.TRUE));
                 break;
             case SUCCESS:
-                mustQueryList.add(QueryBuilders.matchQuery(SegmentRecord.IS_ERROR, BooleanUtils.FALSE));
+                query.must(Query.match(SegmentRecord.IS_ERROR, BooleanUtils.FALSE));
                 break;
         }
+
+        final SearchBuilder search = Search.builder().query(query);
+
         switch (queryOrder) {
             case BY_START_TIME:
-                sourceBuilder.sort(SegmentRecord.START_TIME, SortOrder.DESC);
+                search.sort(SegmentRecord.START_TIME, Sort.Order.DESC);
                 break;
             case BY_DURATION:
-                sourceBuilder.sort(SegmentRecord.LATENCY, SortOrder.DESC);
+                search.sort(SegmentRecord.LATENCY, Sort.Order.DESC);
                 break;
         }
         if (CollectionUtils.isNotEmpty(tags)) {
-            BoolQueryBuilder tagMatchQuery = QueryBuilders.boolQuery();
-            tags.forEach(tag -> tagMatchQuery.must(QueryBuilders.termQuery(SegmentRecord.TAGS, tag.toString())));
-            mustQueryList.add(tagMatchQuery);
+            BoolQueryBuilder tagMatchQuery = Query.bool();
+            tags.forEach(tag -> tagMatchQuery.must(Query.term(SegmentRecord.TAGS, tag)));
+            query.must(tagMatchQuery);
         }
-        sourceBuilder.size(limit);
-        sourceBuilder.from(from);
-        SearchResponse response = getClient().search(
-            new TimeRangeIndexNameMaker(
-                IndexController.LogicIndicesRegister.getPhysicalTableName(SegmentRecord.INDEX_NAME), startSecondTB,
+        search.size(limit).from(from);
+
+        SearchResponse response = null; /*getClient().search(
+            new TimeRangeIndexNameMaker( TODO
+                IndexController.LogicIndicesRegister.getPhysicalTableName(SegmentRecord.INDEX_NAME),
+                startSecondTB,
                 endSecondTB
-            ), sourceBuilder);
+            ), search.build());*/
         TraceBrief traceBrief = new TraceBrief();
         traceBrief.setTotal((int) response.getHits().totalHits);
 
@@ -159,23 +160,26 @@ public class TraceQueryEsDAO extends EsDAO implements ITraceQueryDAO {
 
     @Override
     public List<SegmentRecord> queryByTraceId(String traceId) throws IOException {
-        SearchSourceBuilder sourceBuilder = SearchSourceBuilder.searchSource();
-        sourceBuilder.query(QueryBuilders.termQuery(SegmentRecord.TRACE_ID, traceId));
-        sourceBuilder.size(segmentQueryMaxSize);
+        final String index =
+            IndexController.LogicIndicesRegister.getPhysicalTableName(SegmentRecord.INDEX_NAME);
+
+        final SearchBuilder search =
+            Search.builder()
+                  .query(Query.term(SegmentRecord.TRACE_ID, traceId))
+                  .size(segmentQueryMaxSize);
 
-        SearchResponse response = getClient().search(
-            IndexController.LogicIndicesRegister.getPhysicalTableName(SegmentRecord.INDEX_NAME), sourceBuilder);
+        org.apache.skywalking.library.elasticsearch.response.search.SearchResponse response = getClient().search(index, search.build());
 
         List<SegmentRecord> segmentRecords = new ArrayList<>();
-        for (SearchHit searchHit : response.getHits().getHits()) {
+        for (org.apache.skywalking.library.elasticsearch.response.search.SearchHit searchHit : response.getHits().getHits()) {
             SegmentRecord segmentRecord = new SegmentRecord();
-            segmentRecord.setSegmentId((String) searchHit.getSourceAsMap().get(SegmentRecord.SEGMENT_ID));
-            segmentRecord.setTraceId((String) searchHit.getSourceAsMap().get(SegmentRecord.TRACE_ID));
-            segmentRecord.setServiceId((String) searchHit.getSourceAsMap().get(SegmentRecord.SERVICE_ID));
-            segmentRecord.setStartTime(((Number) searchHit.getSourceAsMap().get(SegmentRecord.START_TIME)).longValue());
-            segmentRecord.setLatency(((Number) searchHit.getSourceAsMap().get(SegmentRecord.LATENCY)).intValue());
-            segmentRecord.setIsError(((Number) searchHit.getSourceAsMap().get(SegmentRecord.IS_ERROR)).intValue());
-            String dataBinaryBase64 = (String) searchHit.getSourceAsMap().get(SegmentRecord.DATA_BINARY);
+            segmentRecord.setSegmentId((String) searchHit.getSource().get(SegmentRecord.SEGMENT_ID));
+            segmentRecord.setTraceId((String) searchHit.getSource().get(SegmentRecord.TRACE_ID));
+            segmentRecord.setServiceId((String) searchHit.getSource().get(SegmentRecord.SERVICE_ID));
+            segmentRecord.setStartTime(((Number) searchHit.getSource().get(SegmentRecord.START_TIME)).longValue());
+            segmentRecord.setLatency(((Number) searchHit.getSource().get(SegmentRecord.LATENCY)).intValue());
+            segmentRecord.setIsError(((Number) searchHit.getSource().get(SegmentRecord.IS_ERROR)).intValue());
+            String dataBinaryBase64 = (String) searchHit.getSource().get(SegmentRecord.DATA_BINARY);
             if (!Strings.isNullOrEmpty(dataBinaryBase64)) {
                 segmentRecord.setDataBinary(Base64.getDecoder().decode(dataBinaryBase64));
             }
diff --git a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/UITemplateManagementEsDAO.java b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/UITemplateManagementEsDAO.java
index 809320c..92a44be 100644
--- a/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/UITemplateManagementEsDAO.java
+++ b/oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/query/UITemplateManagementEsDAO.java
@@ -22,7 +22,15 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.library.elasticsearch.requests.search.BoolQueryBuilder;
+import org.apache.skywalking.library.elasticsearch.requests.search.Query;
+import org.apache.skywalking.library.elasticsearch.requests.search.Search;
+import org.apache.skywalking.library.elasticsearch.requests.search.SearchBuilder;
+import org.apache.skywalking.library.elasticsearch.response.Document;
+import org.apache.skywalking.library.elasticsearch.response.search.SearchHit;
+import org.apache.skywalking.library.elasticsearch.response.search.SearchResponse;
 import org.apache.skywalking.oap.server.core.management.ui.template.UITemplate;
 import org.apache.skywalking.oap.server.core.query.input.DashboardSetting;
 import org.apache.skywalking.oap.server.core.query.type.DashboardConfiguration;
@@ -32,13 +40,6 @@ import org.apache.skywalking.oap.server.library.client.elasticsearch.ElasticSear
 import org.apache.skywalking.oap.server.library.util.BooleanUtils;
 import org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base.EsDAO;
 import org.apache.skywalking.oap.server.storage.plugin.elasticsearch.base.IndexController;
-import org.elasticsearch.action.get.GetResponse;
-import org.elasticsearch.action.search.SearchResponse;
-import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.index.query.BoolQueryBuilder;
-import org.elasticsearch.index.query.QueryBuilders;
-import org.elasticsearch.search.SearchHit;
-import org.elasticsearch.search.builder.SearchSourceBuilder;
 
 @Slf4j
 public class UITemplateManagementEsDAO extends EsDAO implements UITemplateManagementDAO {
@@ -47,29 +48,29 @@ public class UITemplateManagementEsDAO extends EsDAO implements UITemplateManage
     }
 
     @Override
-    public List<DashboardConfiguration> getAllTemplates(final Boolean includingDisabled) throws IOException {
-        SearchSourceBuilder sourceBuilder = SearchSourceBuilder.searchSource();
-
-        BoolQueryBuilder boolQueryBuilder = QueryBuilders.boolQuery();
+    public List<DashboardConfiguration> getAllTemplates(final Boolean includingDisabled)
+        throws IOException {
+        final BoolQueryBuilder boolQuery = Query.bool();
         if (!includingDisabled) {
-            boolQueryBuilder.must()
-                            .add(QueryBuilders.termQuery(
-                                UITemplate.DISABLED,
-                                BooleanUtils.booleanToValue(includingDisabled)
-                            ));
+            boolQuery.must(Query.term(
+                UITemplate.DISABLED,
+                BooleanUtils.booleanToValue(includingDisabled)
+            ));
         }
 
-        sourceBuilder.query(boolQueryBuilder);
-        //It is impossible we have 10000+ templates.
-        sourceBuilder.size(10000);
+        final SearchBuilder search =
+            Search.builder().query(boolQuery)
+                  // It is impossible we have 10000+ templates.
+                  .size(10000);
 
-        SearchResponse response = getClient().search(
-            IndexController.LogicIndicesRegister.getPhysicalTableName(UITemplate.INDEX_NAME), sourceBuilder);
+        final String index =
+            IndexController.LogicIndicesRegister.getPhysicalTableName(UITemplate.INDEX_NAME);
+        final SearchResponse response = getClient().search(index, search.build());
 
-        List<DashboardConfiguration> configs = new ArrayList<>();
+        final List<DashboardConfiguration> configs = new ArrayList<>();
         final UITemplate.Builder builder = new UITemplate.Builder();
         for (SearchHit searchHit : response.getHits()) {
-            Map<String, Object> sourceAsMap = searchHit.getSourceAsMap();
+            Map<String, Object> sourceAsMap = searchHit.getSource();
 
             final UITemplate uiTemplate = builder.storage2Entity(sourceAsMap);
             configs.add(new DashboardConfiguration().fromEntity(uiTemplate));
@@ -83,17 +84,19 @@ public class UITemplateManagementEsDAO extends EsDAO implements UITemplateManage
             final UITemplate.Builder builder = new UITemplate.Builder();
             final UITemplate uiTemplate = setting.toEntity();
 
-            final GetResponse response = getClient().get(UITemplate.INDEX_NAME, uiTemplate.id());
-            if (response.isExists()) {
-                return TemplateChangeStatus.builder().status(false).message("Template exists").build();
+            final boolean exist = getClient().existDoc(UITemplate.INDEX_NAME, uiTemplate.id());
+            if (exist) {
+                return TemplateChangeStatus.builder().status(false).message("Template exists")
+                                           .build();
             }
 
-            XContentBuilder xContentBuilder = map2builder(builder.entity2Storage(uiTemplate));
+            final Map<String, Object> xContentBuilder = builder.entity2Storage(uiTemplate);
             getClient().forceInsert(UITemplate.INDEX_NAME, uiTemplate.id(), xContentBuilder);
             return TemplateChangeStatus.builder().status(true).build();
         } catch (IOException e) {
             log.error(e.getMessage(), e);
-            return TemplateChangeStatus.builder().status(false).message("Can't add a new template").build();
+            return TemplateChangeStatus.builder().status(false).message("Can't add a new template")
+                                       .build();
         }
     }
 
@@ -103,33 +106,36 @@ public class UITemplateManagementEsDAO extends EsDAO implements UITemplateManage
             final UITemplate.Builder builder = new UITemplate.Builder();
             final UITemplate uiTemplate = setting.toEntity();
 
-            final GetResponse response = getClient().get(UITemplate.INDEX_NAME, uiTemplate.id());
-            if (!response.isExists()) {
-                return TemplateChangeStatus.builder().status(false).message("Can't find the template").build();
+            final boolean exist = getClient().existDoc(UITemplate.INDEX_NAME, uiTemplate.id());
+            if (!exist) {
+                return TemplateChangeStatus.builder().status(false)
+                                           .message("Can't find the template").build();
             }
 
-            XContentBuilder xContentBuilder = map2builder(builder.entity2Storage(uiTemplate));
+            final Map<String, Object> xContentBuilder = builder.entity2Storage(uiTemplate);
             getClient().forceUpdate(UITemplate.INDEX_NAME, uiTemplate.id(), xContentBuilder);
             return TemplateChangeStatus.builder().status(true).build();
         } catch (IOException e) {
             log.error(e.getMessage(), e);
-            return TemplateChangeStatus.builder().status(false).message("Can't find the template").build();
+            return TemplateChangeStatus.builder().status(false).message("Can't find the template")
+                                       .build();
         }
     }
 
     @Override
     public TemplateChangeStatus disableTemplate(final String name) throws IOException {
-        final GetResponse response = getClient().get(UITemplate.INDEX_NAME, name);
-        if (response.isExists()) {
+        final Optional<Document> response = getClient().get(UITemplate.INDEX_NAME, name);
+        if (response.isPresent()) {
             final UITemplate.Builder builder = new UITemplate.Builder();
-            final UITemplate uiTemplate = builder.storage2Entity(response.getSourceAsMap());
+            final UITemplate uiTemplate = builder.storage2Entity(response.get().getSource());
             uiTemplate.setDisabled(BooleanUtils.TRUE);
 
-            XContentBuilder xContentBuilder = map2builder(builder.entity2Storage(uiTemplate));
+            final Map<String, Object> xContentBuilder = builder.entity2Storage(uiTemplate);
             getClient().forceUpdate(UITemplate.INDEX_NAME, uiTemplate.id(), xContentBuilder);
             return TemplateChangeStatus.builder().status(true).build();
         } else {
-            return TemplateChangeStatus.builder().status(false).message("Can't find the template").build();
+            return TemplateChangeStatus.builder().status(false).message("Can't find the template")
+                                       .build();
         }
     }
 }
diff --git a/oap-server/server-storage-plugin/storage-elasticsearch7-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch7/client/ElasticSearch7Client.java b/oap-server/server-storage-plugin/storage-elasticsearch7-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch7/client/ElasticSearch7Client.java
index f8b68ca..4d99f60 100644
--- a/oap-server/server-storage-plugin/storage-elasticsearch7-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch7/client/ElasticSearch7Client.java
+++ b/oap-server/server-storage-plugin/storage-elasticsearch7-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch7/client/ElasticSearch7Client.java
@@ -38,6 +38,8 @@ import org.apache.http.HttpHost;
 import org.apache.http.HttpStatus;
 import org.apache.http.client.methods.HttpGet;
 import org.apache.skywalking.apm.util.StringUtil;
+import org.apache.skywalking.library.elasticsearch.response.Document;
+import org.apache.skywalking.library.elasticsearch.response.Documents;
 import org.apache.skywalking.oap.server.library.client.elasticsearch.ElasticSearchClient;
 import org.apache.skywalking.oap.server.library.client.elasticsearch.IndexNameConverter;
 import org.apache.skywalking.oap.server.library.client.request.InsertRequest;
@@ -49,14 +51,11 @@ import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest;
 import org.elasticsearch.action.admin.indices.template.delete.DeleteIndexTemplateRequest;
 import org.elasticsearch.action.bulk.BackoffPolicy;
 import org.elasticsearch.action.bulk.BulkProcessor;
-import org.elasticsearch.action.bulk.BulkRequest;
-import org.elasticsearch.action.bulk.BulkResponse;
 import org.elasticsearch.action.get.GetRequest;
 import org.elasticsearch.action.get.GetResponse;
 import org.elasticsearch.action.index.IndexRequest;
 import org.elasticsearch.action.search.SearchRequest;
 import org.elasticsearch.action.search.SearchResponse;
-import org.elasticsearch.action.support.ActiveShardCount;
 import org.elasticsearch.action.support.IndicesOptions;
 import org.elasticsearch.action.support.WriteRequest;
 import org.elasticsearch.action.support.master.AcknowledgedResponse;
@@ -124,18 +123,6 @@ public class ElasticSearch7Client extends ElasticSearchClient {
         return response.isAcknowledged();
     }
 
-    @Override
-    public boolean createIndex(String indexName, Map<String, Object> settings,
-                               Map<String, Object> mapping) throws IOException {
-        indexName = formatIndexName(indexName);
-        CreateIndexRequest request = new CreateIndexRequest(indexName);
-        request.settings(settings);
-        request.mapping(mapping);
-        CreateIndexResponse response = client.indices().create(request, RequestOptions.DEFAULT);
-        log.debug("create {} index finished, isAcknowledged: {}", indexName, response.isAcknowledged());
-        return response.isAcknowledged();
-    }
-
     /**
      * {@inheritDoc}
      */
@@ -310,7 +297,7 @@ public class ElasticSearch7Client extends ElasticSearchClient {
     }
 
     @Override
-    public GetResponse get(String indexName, String id) throws IOException {
+    public Optional<Document> get(String indexName, String id) throws IOException {
         indexName = formatIndexName(indexName);
         GetRequest request = new GetRequest(indexName, id);
         try {
@@ -324,7 +311,7 @@ public class ElasticSearch7Client extends ElasticSearchClient {
     }
 
     @Override
-    public SearchResponse ids(String indexName, String[] ids) throws IOException {
+    public Optional<Documents> ids(String indexName, String[] ids) throws IOException {
         indexName = formatIndexName(indexName);
 
         SearchRequest searchRequest = new SearchRequest(indexName);
@@ -393,27 +380,8 @@ public class ElasticSearch7Client extends ElasticSearchClient {
         return HttpStatus.SC_OK;
     }
 
-    /**
-     * @since 8.7.0 SkyWalking don't use sync bulk anymore. This method is just kept for unexpected case in the future.
-     */
-    @Deprecated
-    @Override
-    public void synchronousBulk(BulkRequest request) {
-        request.timeout(TimeValue.timeValueMinutes(2));
-        request.setRefreshPolicy(WriteRequest.RefreshPolicy.WAIT_UNTIL);
-        request.waitForActiveShards(ActiveShardCount.ONE);
-        try {
-            int size = request.requests().size();
-            BulkResponse responses = client.bulk(request, RequestOptions.DEFAULT);
-            log.info("Synchronous bulk took time: {} millis, size: {}", responses.getTook().getMillis(), size);
-            healthChecker.health();
-        } catch (Throwable t) {
-            healthChecker.unHealth(t);
-        }
-    }
-
     @Override
-    public BulkProcessor createBulkProcessor(int bulkActions, int flushInterval, int concurrentRequests) {
+    public org.apache.skywalking.library.elasticsearch.bulk.BulkProcessor createBulkProcessor(int bulkActions, int flushInterval, int concurrentRequests) {
         BulkProcessor.Listener listener = createBulkListener();
 
         return BulkProcessor.builder(