You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@skywalking.apache.org by wu...@apache.org on 2018/12/07 14:54:28 UTC

[incubator-skywalking] branch master updated: Fix the NPE about elasticsearch plugin (#1985)

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

wusheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-skywalking.git


The following commit(s) were added to refs/heads/master by this push:
     new aa9a7c4  Fix the NPE about elasticsearch plugin (#1985)
aa9a7c4 is described below

commit aa9a7c443774d96385f308f2ec1f425f47719958
Author: Xin,Zhang <zh...@apache.org>
AuthorDate: Fri Dec 7 22:54:23 2018 +0800

    Fix the NPE about elasticsearch plugin (#1985)
---
 .../v5/ActionRequestBuilderInterceptor.java        | 65 -------------------
 .../v5/AddTransportAddressesInterceptor.java       | 49 ++++++++++++++
 .../apm/plugin/elasticsearch/v5/Constants.java     |  2 -
 .../elasticsearch/v5/ElasticSearchEnhanceInfo.java | 17 +++--
 .../v5/GenericActionConstructorInterceptor.java    | 28 ++++++++
 .../v5/RemoveTransportAddressInterceptor.java      | 49 ++++++++++++++
 .../v5/TransportActionNodeProxyInterceptor.java    | 31 +++++----
 .../elasticsearch/v5/TransportAddressCache.java    | 60 ++++++++++++++++++
 .../v5/TransportProxyClientInterceptor.java        | 64 ++++++++-----------
 ...tion.java => GenericActionInstrumentation.java} | 45 ++++++-------
 .../TransportActionNodeProxyInstrumentation.java   | 20 +++++-
 ...TransportClientNodesServiceInstrumentation.java | 74 ++++++++++++++++++++++
 .../TransportProxyClientInstrumentation.java       | 26 ++------
 .../src/main/resources/skywalking-plugin.def       |  3 +-
 14 files changed, 354 insertions(+), 179 deletions(-)

diff --git a/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/ActionRequestBuilderInterceptor.java b/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/ActionRequestBuilderInterceptor.java
deleted file mode 100644
index 880b364..0000000
--- a/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/ActionRequestBuilderInterceptor.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.skywalking.apm.plugin.elasticsearch.v5;
-
-import org.apache.skywalking.apm.agent.core.context.ContextManager;
-import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.EnhancedInstance;
-import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.InstanceConstructorInterceptor;
-import org.elasticsearch.client.transport.TransportClient;
-
-import static org.apache.skywalking.apm.plugin.elasticsearch.v5.Constants.ES_ENHANCE_INFO;
-
-/**
- * @author oatiz.
- */
-public class ActionRequestBuilderInterceptor implements InstanceConstructorInterceptor {
-
-    @Override
-    public void onConstruct(EnhancedInstance objInst, Object[] allArguments) {
-        ElasticSearchEnhanceInfo enhanceInfo = new ElasticSearchEnhanceInfo();
-
-        parseClientInfo(allArguments[0], enhanceInfo);
-
-        ContextManager.getRuntimeContext().put(ES_ENHANCE_INFO, enhanceInfo);
-    }
-
-    private void parseClientInfo(Object client, ElasticSearchEnhanceInfo enhanceInfo) {
-
-        if (client instanceof TransportClient) {
-            TransportClient transportClient = (TransportClient) client;
-
-            StringBuilder builder = new StringBuilder();
-            for (int i = 0; i < transportClient.transportAddresses().size(); i++) {
-                if (i != transportClient.transportAddresses().size() - 1) {
-                    builder.append(transportClient.transportAddresses().get(i).toString()).append(",");
-                } else {
-                    builder.append(transportClient.transportAddresses().get(i).toString());
-                }
-            }
-
-            enhanceInfo.setTransportAddress(builder.toString());
-        } else {
-            // avoid NPE
-            enhanceInfo.setTransportAddress("");
-        }
-
-
-    }
-
-}
diff --git a/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/AddTransportAddressesInterceptor.java b/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/AddTransportAddressesInterceptor.java
new file mode 100644
index 0000000..0d2d1e1
--- /dev/null
+++ b/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/AddTransportAddressesInterceptor.java
@@ -0,0 +1,49 @@
+/*
+ * 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.apm.plugin.elasticsearch.v5;
+
+import java.lang.reflect.Method;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.EnhancedInstance;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.InstanceMethodsAroundInterceptor;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.MethodInterceptResult;
+import org.elasticsearch.common.transport.TransportAddress;
+
+public class AddTransportAddressesInterceptor implements InstanceMethodsAroundInterceptor {
+    @Override
+    public void beforeMethod(EnhancedInstance objInst, Method method, Object[] allArguments, Class<?>[] argumentsTypes,
+        MethodInterceptResult result) throws Throwable {
+
+    }
+
+    @Override
+    public Object afterMethod(EnhancedInstance objInst, Method method, Object[] allArguments, Class<?>[] argumentsTypes,
+        Object ret) throws Throwable {
+        TransportAddressCache transportAddressCache = (TransportAddressCache)objInst.getSkyWalkingDynamicField();
+        if (transportAddressCache == null) {
+            transportAddressCache = new TransportAddressCache();
+        }
+        transportAddressCache.addDiscoveryNode((TransportAddress[])allArguments[0]);
+        objInst.setSkyWalkingDynamicField(transportAddressCache);
+        return ret;
+    }
+
+    @Override public void handleMethodException(EnhancedInstance objInst, Method method, Object[] allArguments,
+        Class<?>[] argumentsTypes, Throwable t) {
+
+    }
+}
diff --git a/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/Constants.java b/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/Constants.java
index 45aed75..7ac3c7f 100644
--- a/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/Constants.java
+++ b/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/Constants.java
@@ -27,8 +27,6 @@ class Constants {
 
     static final String ELASTICSEARCH_DB_OP_PREFIX = "Elasticsearch/";
 
-    static final String ES_ENHANCE_INFO = "es_enhance_info";
-
     static final String BASE_FUTURE_METHOD = "actionGet";
 
     static final String ES_NODE = "node.address";
diff --git a/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/ElasticSearchEnhanceInfo.java b/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/ElasticSearchEnhanceInfo.java
index 1edcd7b..1d950dc 100644
--- a/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/ElasticSearchEnhanceInfo.java
+++ b/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/ElasticSearchEnhanceInfo.java
@@ -18,15 +18,13 @@
 
 package org.apache.skywalking.apm.plugin.elasticsearch.v5;
 
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.EnhancedInstance;
+
 /**
  * @author oatiz.
  */
 public class ElasticSearchEnhanceInfo {
     /**
-     * elasticsearch host and port
-     */
-    private String transportAddress;
-    /**
      * elasticsearch cluster name
      */
     private String clusterName;
@@ -47,12 +45,10 @@ public class ElasticSearchEnhanceInfo {
      */
     private String source;
 
-    public String getTransportAddress() {
-        return transportAddress;
-    }
+    private EnhancedInstance transportAddressHolder;
 
-    public void setTransportAddress(String transportAddress) {
-        this.transportAddress = transportAddress;
+    public String transportAddresses() {
+        return ((TransportAddressCache)transportAddressHolder.getSkyWalkingDynamicField()).transportAddress();
     }
 
     public String getClusterName() {
@@ -95,4 +91,7 @@ public class ElasticSearchEnhanceInfo {
         this.source = source;
     }
 
+    public void setTransportAddressHolder(EnhancedInstance service) {
+        this.transportAddressHolder = service;
+    }
 }
diff --git a/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/GenericActionConstructorInterceptor.java b/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/GenericActionConstructorInterceptor.java
new file mode 100644
index 0000000..c3fcdf7
--- /dev/null
+++ b/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/GenericActionConstructorInterceptor.java
@@ -0,0 +1,28 @@
+/*
+ * 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.apm.plugin.elasticsearch.v5;
+
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.EnhancedInstance;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.InstanceConstructorInterceptor;
+
+public class GenericActionConstructorInterceptor implements InstanceConstructorInterceptor {
+    @Override
+    public void onConstruct(EnhancedInstance objInst, Object[] allArguments) {
+        //DO Nothing, just for pass parameter
+    }
+}
diff --git a/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/RemoveTransportAddressInterceptor.java b/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/RemoveTransportAddressInterceptor.java
new file mode 100644
index 0000000..5c7835b
--- /dev/null
+++ b/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/RemoveTransportAddressInterceptor.java
@@ -0,0 +1,49 @@
+/*
+ * 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.apm.plugin.elasticsearch.v5;
+
+import java.lang.reflect.Method;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.EnhancedInstance;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.InstanceMethodsAroundInterceptor;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.MethodInterceptResult;
+import org.elasticsearch.common.transport.TransportAddress;
+
+public class RemoveTransportAddressInterceptor implements InstanceMethodsAroundInterceptor {
+    @Override
+    public void beforeMethod(EnhancedInstance objInst, Method method, Object[] allArguments, Class<?>[] argumentsTypes,
+        MethodInterceptResult result) throws Throwable {
+
+    }
+
+    @Override
+    public Object afterMethod(EnhancedInstance objInst, Method method, Object[] allArguments, Class<?>[] argumentsTypes,
+        Object ret) throws Throwable {
+        TransportAddressCache transportAddressCache = (TransportAddressCache)objInst.getSkyWalkingDynamicField();
+        if (transportAddressCache == null) {
+            transportAddressCache = new TransportAddressCache();
+        }
+        transportAddressCache.removeDiscoveryNode((TransportAddress)allArguments[0]);
+        objInst.setSkyWalkingDynamicField(transportAddressCache);
+        return ret;
+    }
+
+    @Override public void handleMethodException(EnhancedInstance objInst, Method method, Object[] allArguments,
+        Class<?>[] argumentsTypes, Throwable t) {
+
+    }
+}
diff --git a/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/TransportActionNodeProxyInterceptor.java b/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/TransportActionNodeProxyInterceptor.java
index 9f3a866..adc431c 100644
--- a/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/TransportActionNodeProxyInterceptor.java
+++ b/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/TransportActionNodeProxyInterceptor.java
@@ -18,60 +18,67 @@
 
 package org.apache.skywalking.apm.plugin.elasticsearch.v5;
 
+import java.lang.reflect.Method;
 import org.apache.skywalking.apm.agent.core.context.ContextManager;
 import org.apache.skywalking.apm.agent.core.context.tag.Tags;
 import org.apache.skywalking.apm.agent.core.context.trace.AbstractSpan;
 import org.apache.skywalking.apm.agent.core.context.trace.SpanLayer;
 import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.EnhancedInstance;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.InstanceConstructorInterceptor;
 import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.InstanceMethodsAroundInterceptor;
 import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.MethodInterceptResult;
 import org.apache.skywalking.apm.network.trace.component.ComponentsDefine;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 
-import java.lang.reflect.Method;
-
 import static org.apache.skywalking.apm.agent.core.conf.Config.Plugin.Elasticsearch.TRACE_DSL;
-import static org.apache.skywalking.apm.plugin.elasticsearch.v5.Constants.*;
+import static org.apache.skywalking.apm.plugin.elasticsearch.v5.Constants.DB_TYPE;
+import static org.apache.skywalking.apm.plugin.elasticsearch.v5.Constants.ELASTICSEARCH_DB_OP_PREFIX;
+import static org.apache.skywalking.apm.plugin.elasticsearch.v5.Constants.ES_INDEX;
+import static org.apache.skywalking.apm.plugin.elasticsearch.v5.Constants.ES_NODE;
+import static org.apache.skywalking.apm.plugin.elasticsearch.v5.Constants.ES_TYPE;
 import static org.apache.skywalking.apm.plugin.elasticsearch.v5.Util.wrapperNullStringValue;
 
 /**
  * @author oatiz.
  */
-public class TransportActionNodeProxyInterceptor implements InstanceMethodsAroundInterceptor {
+public class TransportActionNodeProxyInterceptor implements InstanceConstructorInterceptor, InstanceMethodsAroundInterceptor {
 
     @Override
     public void beforeMethod(EnhancedInstance objInst, Method method, Object[] allArguments,
-                             Class<?>[] argumentsTypes, MethodInterceptResult result) throws Throwable {
-
-        ElasticSearchEnhanceInfo enhanceInfo = (ElasticSearchEnhanceInfo) ContextManager.getRuntimeContext().get(ES_ENHANCE_INFO);
+        Class<?>[] argumentsTypes, MethodInterceptResult result) throws Throwable {
 
+        ElasticSearchEnhanceInfo enhanceInfo = (ElasticSearchEnhanceInfo)((EnhancedInstance)objInst.getSkyWalkingDynamicField()).getSkyWalkingDynamicField();
         String opType = allArguments[1].getClass().getSimpleName();
         String operationName = ELASTICSEARCH_DB_OP_PREFIX + opType;
-        AbstractSpan span = ContextManager.createExitSpan(operationName, enhanceInfo.getTransportAddress());
+        AbstractSpan span = ContextManager.createExitSpan(operationName, enhanceInfo.transportAddresses());
         span.setComponent(ComponentsDefine.TRANSPORT_CLIENT);
         Tags.DB_TYPE.set(span, DB_TYPE);
         Tags.DB_INSTANCE.set(span, enhanceInfo.getClusterName());
         if (TRACE_DSL) {
             Tags.DB_STATEMENT.set(span, enhanceInfo.getSource());
         }
-        span.tag(ES_NODE, ((DiscoveryNode) allArguments[0]).getAddress().toString());
+        span.tag(ES_NODE, ((DiscoveryNode)allArguments[0]).getAddress().toString());
         span.tag(ES_INDEX, wrapperNullStringValue(enhanceInfo.getIndices()));
         span.tag(ES_TYPE, wrapperNullStringValue(enhanceInfo.getTypes()));
         SpanLayer.asDB(span);
-        ContextManager.getRuntimeContext().remove(ES_ENHANCE_INFO);
     }
 
     @Override
     public Object afterMethod(EnhancedInstance objInst, Method method, Object[] allArguments,
-                              Class<?>[] argumentsTypes, Object ret) throws Throwable {
+        Class<?>[] argumentsTypes, Object ret) throws Throwable {
         ContextManager.stopSpan();
         return ret;
     }
 
     @Override
     public void handleMethodException(EnhancedInstance objInst, Method method, Object[] allArguments,
-                                      Class<?>[] argumentsTypes, Throwable t) {
+        Class<?>[] argumentsTypes, Throwable t) {
         ContextManager.activeSpan().errorOccurred().log(t);
     }
 
+    @Override
+    public void onConstruct(EnhancedInstance objInst, Object[] allArguments) {
+        EnhancedInstance actions = (EnhancedInstance)allArguments[1];
+        objInst.setSkyWalkingDynamicField(actions);
+    }
 }
diff --git a/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/TransportAddressCache.java b/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/TransportAddressCache.java
new file mode 100644
index 0000000..725dcb6
--- /dev/null
+++ b/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/TransportAddressCache.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.skywalking.apm.plugin.elasticsearch.v5;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.elasticsearch.common.transport.TransportAddress;
+
+public class TransportAddressCache {
+
+    private List<TransportAddress> transportAddresses = new ArrayList<TransportAddress>();
+    private String transportAddressesStr;
+
+    public synchronized void addDiscoveryNode(TransportAddress... transportAddress) {
+        transportAddresses.addAll(Arrays.asList(transportAddress));
+        transportAddressesStr = format();
+    }
+
+    public synchronized void removeDiscoveryNode(TransportAddress transportAddress) {
+        List<TransportAddress> nodesBuilder = new ArrayList<TransportAddress>();
+
+        for (TransportAddress otherNode : transportAddresses) {
+            if (!otherNode.getAddress().equals(transportAddress.getAddress())) {
+                nodesBuilder.add(otherNode);
+            }
+        }
+
+        transportAddresses = nodesBuilder;
+        transportAddressesStr = format();
+    }
+
+    private String format() {
+        StringBuilder stringBuilder = new StringBuilder();
+        for (TransportAddress node : transportAddresses) {
+            stringBuilder.append(node.getAddress()).append(":").append(node.getPort()).append(";");
+        }
+
+        return stringBuilder.toString();
+    }
+
+    public String transportAddress() {
+        return transportAddressesStr;
+    }
+}
diff --git a/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/TransportProxyClientInterceptor.java b/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/TransportProxyClientInterceptor.java
index 76b713a..6513001 100644
--- a/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/TransportProxyClientInterceptor.java
+++ b/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/TransportProxyClientInterceptor.java
@@ -6,26 +6,25 @@
  * (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
+ *      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.
- *
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
  */
 
 package org.apache.skywalking.apm.plugin.elasticsearch.v5;
 
-import org.apache.skywalking.apm.agent.core.context.ContextManager;
+import java.io.IOException;
+import java.util.List;
 import org.apache.skywalking.apm.agent.core.logging.api.ILog;
 import org.apache.skywalking.apm.agent.core.logging.api.LogManager;
 import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.EnhancedInstance;
 import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.InstanceConstructorInterceptor;
-import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.InstanceMethodsAroundInterceptor;
-import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.MethodInterceptResult;
 import org.apache.skywalking.apm.util.StringUtil;
+import org.elasticsearch.action.GenericAction;
 import org.elasticsearch.action.delete.DeleteRequest;
 import org.elasticsearch.action.get.GetRequest;
 import org.elasticsearch.action.index.IndexRequest;
@@ -34,45 +33,32 @@ import org.elasticsearch.action.update.UpdateRequest;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.xcontent.XContentFactory;
 
-import java.io.IOException;
-import java.lang.reflect.Method;
-
 import static org.apache.skywalking.apm.agent.core.conf.Config.Plugin.Elasticsearch.TRACE_DSL;
-import static org.apache.skywalking.apm.plugin.elasticsearch.v5.Constants.ES_ENHANCE_INFO;
-import static org.apache.skywalking.apm.plugin.elasticsearch.v5.Util.wrapperNullStringValue;
 
 /**
- * @author oatiz.
+ * @author oatiz, zhang xin.
  */
-public class TransportProxyClientInterceptor implements InstanceConstructorInterceptor, InstanceMethodsAroundInterceptor {
+public class TransportProxyClientInterceptor implements InstanceConstructorInterceptor {
 
     private static final ILog logger = LogManager.getLogger(TransportProxyClientInterceptor.class);
 
     @Override
     public void onConstruct(EnhancedInstance objInst, Object[] allArguments) {
-        Settings settings = (Settings) allArguments[0];
+        Settings settings = (Settings)allArguments[0];
         String clusterName = settings.get("cluster.name");
-        objInst.setSkyWalkingDynamicField(wrapperNullStringValue(clusterName));
-    }
 
-    @Override
-    public void beforeMethod(EnhancedInstance objInst, Method method, Object[] allArguments,
-                             Class<?>[] argumentsTypes, MethodInterceptResult result) throws Throwable {
-        ElasticSearchEnhanceInfo enhanceInfo = (ElasticSearchEnhanceInfo) ContextManager.getRuntimeContext().get(ES_ENHANCE_INFO);
-        enhanceInfo.setClusterName((String) objInst.getSkyWalkingDynamicField());
-        parseRequestInfo(allArguments[1], enhanceInfo);
-    }
+        EnhancedInstance nodeService = (EnhancedInstance)allArguments[2];
+        List<GenericAction> genericActions = (List<GenericAction>)allArguments[3];
 
-    @Override
-    public Object afterMethod(EnhancedInstance objInst, Method method, Object[] allArguments,
-                              Class<?>[] argumentsTypes, Object ret) throws Throwable {
-        return ret;
-    }
-
-    @Override
-    public void handleMethodException(EnhancedInstance objInst, Method method, Object[] allArguments,
-                                      Class<?>[] argumentsTypes, Throwable t) {
-        ContextManager.activeSpan().errorOccurred().log(t);
+        for (GenericAction action : genericActions) {
+            if (action instanceof EnhancedInstance) {
+                ElasticSearchEnhanceInfo elasticSearchEnhanceInfo = new ElasticSearchEnhanceInfo();
+                elasticSearchEnhanceInfo.setClusterName(clusterName);
+                parseRequestInfo(action, elasticSearchEnhanceInfo);
+                elasticSearchEnhanceInfo.setTransportAddressHolder(nodeService);
+                ((EnhancedInstance)action).setSkyWalkingDynamicField(elasticSearchEnhanceInfo);
+            }
+        }
     }
 
     private void parseRequestInfo(Object request, ElasticSearchEnhanceInfo enhanceInfo) {
@@ -103,7 +89,7 @@ public class TransportProxyClientInterceptor implements InstanceConstructorInter
     }
 
     private void parseSearchRequest(Object request, ElasticSearchEnhanceInfo enhanceInfo) {
-        SearchRequest searchRequest = (SearchRequest) request;
+        SearchRequest searchRequest = (SearchRequest)request;
         enhanceInfo.setIndices(StringUtil.join(',', searchRequest.indices()));
         enhanceInfo.setTypes(StringUtil.join(',', searchRequest.types()));
         if (TRACE_DSL) {
@@ -112,7 +98,7 @@ public class TransportProxyClientInterceptor implements InstanceConstructorInter
     }
 
     private void parseGetRequest(Object request, ElasticSearchEnhanceInfo enhanceInfo) {
-        GetRequest getRequest = (GetRequest) request;
+        GetRequest getRequest = (GetRequest)request;
         enhanceInfo.setIndices(StringUtil.join(',', getRequest.indices()));
         enhanceInfo.setTypes(getRequest.type());
         if (TRACE_DSL) {
@@ -121,7 +107,7 @@ public class TransportProxyClientInterceptor implements InstanceConstructorInter
     }
 
     private void parseIndexRequest(Object request, ElasticSearchEnhanceInfo enhanceInfo) {
-        IndexRequest indexRequest = (IndexRequest) request;
+        IndexRequest indexRequest = (IndexRequest)request;
         enhanceInfo.setIndices(StringUtil.join(',', indexRequest.indices()));
         enhanceInfo.setTypes(indexRequest.type());
         if (TRACE_DSL) {
@@ -130,7 +116,7 @@ public class TransportProxyClientInterceptor implements InstanceConstructorInter
     }
 
     private void parseUpdateRequest(Object request, ElasticSearchEnhanceInfo enhanceInfo) {
-        UpdateRequest updateRequest = (UpdateRequest) request;
+        UpdateRequest updateRequest = (UpdateRequest)request;
         enhanceInfo.setIndices(StringUtil.join(',', updateRequest.indices()));
         enhanceInfo.setTypes(updateRequest.type());
         if (TRACE_DSL) {
@@ -145,7 +131,7 @@ public class TransportProxyClientInterceptor implements InstanceConstructorInter
     }
 
     private void parseDeleteRequest(Object request, ElasticSearchEnhanceInfo enhanceInfo) {
-        DeleteRequest deleteRequest = (DeleteRequest) request;
+        DeleteRequest deleteRequest = (DeleteRequest)request;
         enhanceInfo.setIndices(StringUtil.join(',', deleteRequest.indices()));
         enhanceInfo.setTypes(deleteRequest.type());
         if (TRACE_DSL) {
diff --git a/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/define/ActionRequestBuilderInstrumentation.java b/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/define/GenericActionInstrumentation.java
similarity index 55%
rename from apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/define/ActionRequestBuilderInstrumentation.java
rename to apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/define/GenericActionInstrumentation.java
index b10c0f2..9bb444b 100644
--- a/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/define/ActionRequestBuilderInstrumentation.java
+++ b/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/define/GenericActionInstrumentation.java
@@ -6,14 +6,13 @@
  * (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
+ *      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.
- *
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
  */
 
 package org.apache.skywalking.apm.plugin.elasticsearch.v5.define;
@@ -22,43 +21,37 @@ import net.bytebuddy.description.method.MethodDescription;
 import net.bytebuddy.matcher.ElementMatcher;
 import org.apache.skywalking.apm.agent.core.plugin.interceptor.ConstructorInterceptPoint;
 import org.apache.skywalking.apm.agent.core.plugin.interceptor.InstanceMethodsInterceptPoint;
-import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.ClassInstanceMethodsEnhancePluginDefine;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.StaticMethodsInterceptPoint;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.ClassEnhancePluginDefine;
 import org.apache.skywalking.apm.agent.core.plugin.match.ClassMatch;
 
 import static net.bytebuddy.matcher.ElementMatchers.any;
-import static org.apache.skywalking.apm.agent.core.plugin.match.NameMatch.byName;
-
-/**
- * @author oatiz.
- */
-public class ActionRequestBuilderInstrumentation extends ClassInstanceMethodsEnhancePluginDefine {
+import static org.apache.skywalking.apm.agent.core.plugin.match.HierarchyMatch.byHierarchyMatch;
 
-    private static final String ENHANCE_CLASS = "org.apache.skywalking.apm.plugin.elasticsearch.v5.ActionRequestBuilderInterceptor";
-
-    @Override
-    protected ConstructorInterceptPoint[] getConstructorsInterceptPoints() {
-        return new ConstructorInterceptPoint[]{
+public class GenericActionInstrumentation extends ClassEnhancePluginDefine {
+    @Override protected ConstructorInterceptPoint[] getConstructorsInterceptPoints() {
+        return new ConstructorInterceptPoint[] {
             new ConstructorInterceptPoint() {
-                @Override
-                public ElementMatcher<MethodDescription> getConstructorMatcher() {
+                @Override public ElementMatcher<MethodDescription> getConstructorMatcher() {
                     return any();
                 }
 
-                @Override
-                public String getConstructorInterceptor() {
-                    return ENHANCE_CLASS;
+                @Override public String getConstructorInterceptor() {
+                    return "org.apache.skywalking.apm.plugin.elasticsearch.v5.GenericActionConstructorInterceptor";
                 }
             }
         };
     }
 
-    @Override
-    protected InstanceMethodsInterceptPoint[] getInstanceMethodsInterceptPoints() {
+    @Override protected InstanceMethodsInterceptPoint[] getInstanceMethodsInterceptPoints() {
         return new InstanceMethodsInterceptPoint[0];
     }
 
-    @Override
-    protected ClassMatch enhanceClass() {
-        return byName("org.elasticsearch.action.ActionRequestBuilder");
+    @Override protected StaticMethodsInterceptPoint[] getStaticMethodsInterceptPoints() {
+        return new StaticMethodsInterceptPoint[0];
+    }
+
+    @Override protected ClassMatch enhanceClass() {
+        return byHierarchyMatch(new String[] {"org.elasticsearch.action.GenericAction"});
     }
 }
diff --git a/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/define/TransportActionNodeProxyInstrumentation.java b/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/define/TransportActionNodeProxyInstrumentation.java
index 9eb6951..d4b28a7 100644
--- a/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/define/TransportActionNodeProxyInstrumentation.java
+++ b/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/define/TransportActionNodeProxyInstrumentation.java
@@ -26,6 +26,7 @@ import org.apache.skywalking.apm.agent.core.plugin.interceptor.StaticMethodsInte
 import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.ClassEnhancePluginDefine;
 import org.apache.skywalking.apm.agent.core.plugin.match.ClassMatch;
 
+import static net.bytebuddy.matcher.ElementMatchers.any;
 import static net.bytebuddy.matcher.ElementMatchers.named;
 import static org.apache.skywalking.apm.agent.core.plugin.match.NameMatch.byName;
 
@@ -34,9 +35,22 @@ import static org.apache.skywalking.apm.agent.core.plugin.match.NameMatch.byName
  */
 public class TransportActionNodeProxyInstrumentation extends ClassEnhancePluginDefine {
 
+    public static final String INTERCEPTOR_CLASS = "org.apache.skywalking.apm.plugin.elasticsearch.v5.TransportActionNodeProxyInterceptor";
+    public static final String ENHANC_CLASS = "org.elasticsearch.action.TransportActionNodeProxy";
+
     @Override
     protected ConstructorInterceptPoint[] getConstructorsInterceptPoints() {
-        return new ConstructorInterceptPoint[0];
+        return new ConstructorInterceptPoint[]{
+            new ConstructorInterceptPoint() {
+                @Override public ElementMatcher<MethodDescription> getConstructorMatcher() {
+                    return any();
+                }
+
+                @Override public String getConstructorInterceptor() {
+                    return INTERCEPTOR_CLASS;
+                }
+            }
+        };
     }
 
     @Override
@@ -50,7 +64,7 @@ public class TransportActionNodeProxyInstrumentation extends ClassEnhancePluginD
 
                 @Override
                 public String getMethodsInterceptor() {
-                    return "org.apache.skywalking.apm.plugin.elasticsearch.v5.TransportActionNodeProxyInterceptor";
+                    return INTERCEPTOR_CLASS;
                 }
 
                 @Override
@@ -68,6 +82,6 @@ public class TransportActionNodeProxyInstrumentation extends ClassEnhancePluginD
 
     @Override
     protected ClassMatch enhanceClass() {
-        return byName("org.elasticsearch.action.TransportActionNodeProxy");
+        return byName(ENHANC_CLASS);
     }
 }
diff --git a/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/define/TransportClientNodesServiceInstrumentation.java b/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/define/TransportClientNodesServiceInstrumentation.java
new file mode 100644
index 0000000..243fe32
--- /dev/null
+++ b/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/define/TransportClientNodesServiceInstrumentation.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.apm.plugin.elasticsearch.v5.define;
+
+import net.bytebuddy.description.method.MethodDescription;
+import net.bytebuddy.matcher.ElementMatcher;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.ConstructorInterceptPoint;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.InstanceMethodsInterceptPoint;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.ClassInstanceMethodsEnhancePluginDefine;
+import org.apache.skywalking.apm.agent.core.plugin.match.ClassMatch;
+
+import static net.bytebuddy.matcher.ElementMatchers.named;
+import static org.apache.skywalking.apm.agent.core.plugin.match.NameMatch.byName;
+
+public class TransportClientNodesServiceInstrumentation extends ClassInstanceMethodsEnhancePluginDefine {
+
+    public static final String ADD_TRANSPORT_ADDRESSES_INTERCEPTOR = "org.apache.skywalking.apm.plugin.elasticsearch.v5.AddTransportAddressesInterceptor";
+    public static final String REMOVE_TRANSPORT_ADDRESS_INTERCEPTOR = "org.apache.skywalking.apm.plugin.elasticsearch.v5.RemoveTransportAddressInterceptor";
+    public static final String ENHANCE_CLASS = "org.elasticsearch.client.transport.TransportClientNodesService";
+
+    @Override protected ConstructorInterceptPoint[] getConstructorsInterceptPoints() {
+        return new ConstructorInterceptPoint[0];
+    }
+
+    @Override protected InstanceMethodsInterceptPoint[] getInstanceMethodsInterceptPoints() {
+        return new InstanceMethodsInterceptPoint[]{
+            new InstanceMethodsInterceptPoint() {
+                @Override public ElementMatcher<MethodDescription> getMethodsMatcher() {
+                    return named("addTransportAddresses");
+                }
+
+                @Override public String getMethodsInterceptor() {
+                    return ADD_TRANSPORT_ADDRESSES_INTERCEPTOR;
+                }
+
+                @Override public boolean isOverrideArgs() {
+                    return false;
+                }
+            },
+            new InstanceMethodsInterceptPoint() {
+                @Override public ElementMatcher<MethodDescription> getMethodsMatcher() {
+                    return named("removeTransportAddress");
+                }
+
+                @Override public String getMethodsInterceptor() {
+                    return REMOVE_TRANSPORT_ADDRESS_INTERCEPTOR;
+                }
+
+                @Override public boolean isOverrideArgs() {
+                    return false;
+                }
+            }
+        };
+    }
+
+    @Override protected ClassMatch enhanceClass() {
+        return byName(ENHANCE_CLASS);
+    }
+}
diff --git a/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/define/TransportProxyClientInstrumentation.java b/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/define/TransportProxyClientInstrumentation.java
index b3fa40e..5cb67e6 100644
--- a/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/define/TransportProxyClientInstrumentation.java
+++ b/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/elasticsearch/v5/define/TransportProxyClientInstrumentation.java
@@ -23,23 +23,22 @@ import net.bytebuddy.matcher.ElementMatcher;
 import org.apache.skywalking.apm.agent.core.plugin.interceptor.ConstructorInterceptPoint;
 import org.apache.skywalking.apm.agent.core.plugin.interceptor.InstanceMethodsInterceptPoint;
 import org.apache.skywalking.apm.agent.core.plugin.interceptor.StaticMethodsInterceptPoint;
-import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.ClassEnhancePluginDefine;
+import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.ClassInstanceMethodsEnhancePluginDefine;
 import org.apache.skywalking.apm.agent.core.plugin.match.ClassMatch;
 
 import static net.bytebuddy.matcher.ElementMatchers.any;
-import static net.bytebuddy.matcher.ElementMatchers.named;
 import static org.apache.skywalking.apm.agent.core.plugin.match.NameMatch.byName;
 
 /**
  * @author oatiz.
  */
-public class TransportProxyClientInstrumentation extends ClassEnhancePluginDefine {
+public class TransportProxyClientInstrumentation extends ClassInstanceMethodsEnhancePluginDefine {
 
     private static final String ENHANCE_CLASS = "org.apache.skywalking.apm.plugin.elasticsearch.v5.TransportProxyClientInterceptor";
 
     @Override
     protected ConstructorInterceptPoint[] getConstructorsInterceptPoints() {
-        return new ConstructorInterceptPoint[]{
+        return new ConstructorInterceptPoint[] {
             new ConstructorInterceptPoint() {
                 @Override
                 public ElementMatcher<MethodDescription> getConstructorMatcher() {
@@ -56,24 +55,7 @@ public class TransportProxyClientInstrumentation extends ClassEnhancePluginDefin
 
     @Override
     protected InstanceMethodsInterceptPoint[] getInstanceMethodsInterceptPoints() {
-        return new InstanceMethodsInterceptPoint[]{
-            new InstanceMethodsInterceptPoint() {
-                @Override
-                public ElementMatcher<MethodDescription> getMethodsMatcher() {
-                    return named("execute");
-                }
-
-                @Override
-                public String getMethodsInterceptor() {
-                    return ENHANCE_CLASS;
-                }
-
-                @Override
-                public boolean isOverrideArgs() {
-                    return false;
-                }
-            }
-        };
+        return new InstanceMethodsInterceptPoint[0];
     }
 
     @Override
diff --git a/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/resources/skywalking-plugin.def b/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/resources/skywalking-plugin.def
index 7cb6d6c..d031d4e 100644
--- a/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/resources/skywalking-plugin.def
+++ b/apm-sniffer/apm-sdk-plugin/elasticsearch-5.x-plugin/src/main/resources/skywalking-plugin.def
@@ -14,7 +14,8 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-elasticsearch-5.x=org.apache.skywalking.apm.plugin.elasticsearch.v5.define.ActionRequestBuilderInstrumentation
 elasticsearch-5.x=org.apache.skywalking.apm.plugin.elasticsearch.v5.define.TransportProxyClientInstrumentation
 elasticsearch-5.x=org.apache.skywalking.apm.plugin.elasticsearch.v5.define.TransportActionNodeProxyInstrumentation
 elasticsearch-5.x=org.apache.skywalking.apm.plugin.elasticsearch.v5.define.PlainListenableActionFutureInstrumentation
+elasticsearch-5.x=org.apache.skywalking.apm.plugin.elasticsearch.v5.define.GenericActionInstrumentation
+elasticsearch-5.x=org.apache.skywalking.apm.plugin.elasticsearch.v5.define.TransportClientNodesServiceInstrumentation