You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by da...@apache.org on 2018/08/25 08:09:19 UTC

lucene-solr:jira/http2: SOLR-12643: Adding metrics support for Http2SolrClient

Repository: lucene-solr
Updated Branches:
  refs/heads/jira/http2 56c44139c -> ab28046f2


SOLR-12643: Adding metrics support for Http2SolrClient


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/ab28046f
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/ab28046f
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/ab28046f

Branch: refs/heads/jira/http2
Commit: ab28046f24ceaea5e57f8c9d9b34f785f5432964
Parents: 56c4413
Author: Cao Manh Dat <da...@apache.org>
Authored: Sat Aug 25 15:07:23 2018 +0700
Committer: Cao Manh Dat <da...@apache.org>
Committed: Sat Aug 25 15:07:23 2018 +0700

----------------------------------------------------------------------
 .../component/HttpShardHandlerFactory.java      |  20 ++--
 .../apache/solr/update/UpdateShardHandler.java  |  51 ++++++---
 .../stats/InstrumentedHttpListenerFactory.java  | 114 +++++++++++++++++++
 .../solr/client/solrj/impl/Http2SolrClient.java |  10 ++
 .../client/solrj/impl/HttpListenerFactory.java  |  30 +++++
 5 files changed, 195 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ab28046f/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
index ec59778..4d1666b 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
@@ -59,13 +59,13 @@ import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.update.UpdateShardHandlerConfig;
 import org.apache.solr.util.DefaultSolrThreadFactory;
 import org.apache.solr.util.stats.HttpClientMetricNameStrategy;
-import org.apache.solr.util.stats.InstrumentedHttpRequestExecutor;
+import org.apache.solr.util.stats.InstrumentedHttpListenerFactory;
 import org.apache.solr.util.stats.InstrumentedPoolingHttpClientConnectionManager;
 import org.apache.solr.util.stats.MetricUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.solr.util.stats.InstrumentedHttpRequestExecutor.KNOWN_METRIC_NAME_STRATEGIES;
+import static org.apache.solr.util.stats.InstrumentedHttpListenerFactory.KNOWN_METRIC_NAME_STRATEGIES;
 
 
 public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.apache.solr.util.plugin.PluginInfoInitialized, SolrMetricProducer {
@@ -89,9 +89,8 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
       false
   );
 
-  protected InstrumentedPoolingHttpClientConnectionManager clientConnectionManager;
   protected Http2SolrClient defaultClient;
-  protected InstrumentedHttpRequestExecutor httpRequestExecutor;
+  protected InstrumentedHttpListenerFactory httpListenerFactory;
   private LBHttp2SolrClient loadbalancer;
   //default values:
   int soTimeout = UpdateShardHandlerConfig.DEFAULT_DISTRIBUPDATESOTIMEOUT;
@@ -108,7 +107,7 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
 
   private String scheme = null;
 
-  private HttpClientMetricNameStrategy metricNameStrategy;
+  private InstrumentedHttpListenerFactory.NameStrategy metricNameStrategy;
 
   private String metricTag;
 
@@ -210,10 +209,9 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
         new DefaultSolrThreadFactory("httpShardExecutor")
     );
 
-    //TODO no commit, replace all metrics
-    httpRequestExecutor = new InstrumentedHttpRequestExecutor(this.metricNameStrategy);
-    clientConnectionManager = new InstrumentedPoolingHttpClientConnectionManager(HttpClientUtil.getSchemaRegisteryProvider().getSchemaRegistry());
+    this.httpListenerFactory = new InstrumentedHttpListenerFactory(this.metricNameStrategy);
     this.defaultClient = new Http2SolrClient.Builder().connectionTimeout(connectionTimeout).idleTimeout(soTimeout).build();
+    this.defaultClient.setListenerFactory(this.httpListenerFactory);
     this.loadbalancer = new LBHttp2SolrClient(defaultClient);
   }
 
@@ -255,9 +253,6 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
         if (defaultClient != null) {
           IOUtils.closeQuietly(defaultClient);
         }
-        if (clientConnectionManager != null)  {
-          clientConnectionManager.close();
-        }
       }
     }
   }
@@ -469,8 +464,7 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
   public void initializeMetrics(SolrMetricManager manager, String registry, String tag, String scope) {
     this.metricTag = tag;
     String expandedScope = SolrMetricManager.mkName(scope, SolrInfoBean.Category.QUERY.name());
-    clientConnectionManager.initializeMetrics(manager, registry, tag, expandedScope);
-    httpRequestExecutor.initializeMetrics(manager, registry, tag, expandedScope);
+    httpListenerFactory.initializeMetrics(manager, registry, tag, expandedScope);
     commExecutor = MetricUtils.instrumentedExecutorService(commExecutor, null,
         manager.registry(registry),
         SolrMetricManager.mkName("httpShardExecutor", expandedScope, "threadPool"));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ab28046f/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java b/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
index 19b09c1..d1a8083 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
@@ -39,6 +39,7 @@ import org.apache.solr.core.SolrInfoBean;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.metrics.SolrMetricProducer;
 import org.apache.solr.util.stats.HttpClientMetricNameStrategy;
+import org.apache.solr.util.stats.InstrumentedHttpListenerFactory;
 import org.apache.solr.util.stats.InstrumentedHttpRequestExecutor;
 import org.apache.solr.util.stats.InstrumentedPoolingHttpClientConnectionManager;
 import org.apache.solr.util.stats.MetricUtils;
@@ -70,12 +71,12 @@ public class UpdateShardHandler implements SolrMetricProducer, SolrInfoBean {
   
   private final CloseableHttpClient defaultClient;
 
-  private final InstrumentedPoolingHttpClientConnectionManager updateOnlyConnectionManager;
-  
   private final InstrumentedPoolingHttpClientConnectionManager defaultConnectionManager;
 
   private final InstrumentedHttpRequestExecutor httpRequestExecutor;
 
+  private final InstrumentedHttpListenerFactory updateHttpListenerFactory;
+
 
   private final Set<String> metricNames = ConcurrentHashMap.newKeySet();
   private MetricRegistry registry;
@@ -84,11 +85,8 @@ public class UpdateShardHandler implements SolrMetricProducer, SolrInfoBean {
   private int connectionTimeout = UpdateShardHandlerConfig.DEFAULT_DISTRIBUPDATECONNTIMEOUT;
 
   public UpdateShardHandler(UpdateShardHandlerConfig cfg) {
-    updateOnlyConnectionManager = new InstrumentedPoolingHttpClientConnectionManager(HttpClientUtil.getSchemaRegisteryProvider().getSchemaRegistry());
     defaultConnectionManager = new InstrumentedPoolingHttpClientConnectionManager(HttpClientUtil.getSchemaRegisteryProvider().getSchemaRegistry());
     if (cfg != null ) {
-      updateOnlyConnectionManager.setMaxTotal(cfg.getMaxUpdateConnections());
-      updateOnlyConnectionManager.setDefaultMaxPerRoute(cfg.getMaxUpdateConnectionsPerHost());
       defaultConnectionManager.setMaxTotal(cfg.getMaxUpdateConnections());
       defaultConnectionManager.setDefaultMaxPerRoute(cfg.getMaxUpdateConnectionsPerHost());
     }
@@ -100,23 +98,18 @@ public class UpdateShardHandler implements SolrMetricProducer, SolrInfoBean {
       socketTimeout = cfg.getDistributedSocketTimeout();
       connectionTimeout = cfg.getDistributedConnectionTimeout();
     }
-    HttpClientMetricNameStrategy metricNameStrategy = KNOWN_METRIC_NAME_STRATEGIES.get(UpdateShardHandlerConfig.DEFAULT_METRICNAMESTRATEGY);
-    if (cfg != null)  {
-      metricNameStrategy = KNOWN_METRIC_NAME_STRATEGIES.get(cfg.getMetricNameStrategy());
-      if (metricNameStrategy == null) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
-            "Unknown metricNameStrategy: " + cfg.getMetricNameStrategy() + " found. Must be one of: " + KNOWN_METRIC_NAME_STRATEGIES.keySet());
-      }
-    }
 
+    httpRequestExecutor = new InstrumentedHttpRequestExecutor(getMetricNameStrategy(cfg));
+    updateHttpListenerFactory = new InstrumentedHttpListenerFactory(getNameStrategy(cfg));
 
-    httpRequestExecutor = new InstrumentedHttpRequestExecutor(metricNameStrategy);
     Http2SolrClient.Builder updateOnlyClientBuilder = new Http2SolrClient.Builder();
     if (cfg != null) {
       updateOnlyClientBuilder.connectionTimeout(cfg.getDistributedConnectionTimeout())
           .idleTimeout(cfg.getDistributedSocketTimeout());
     }
     updateOnlyClient = updateOnlyClientBuilder.build();
+    updateOnlyClient.setListenerFactory(updateHttpListenerFactory);
+
     defaultClient = HttpClientUtil.createClient(clientParams, defaultConnectionManager, false, httpRequestExecutor);
 
     // following is done only for logging complete configuration.
@@ -138,6 +131,32 @@ public class UpdateShardHandler implements SolrMetricProducer, SolrInfoBean {
     }
   }
 
+  private HttpClientMetricNameStrategy getMetricNameStrategy(UpdateShardHandlerConfig cfg) {
+    HttpClientMetricNameStrategy metricNameStrategy = KNOWN_METRIC_NAME_STRATEGIES.get(UpdateShardHandlerConfig.DEFAULT_METRICNAMESTRATEGY);
+    if (cfg != null)  {
+      metricNameStrategy = KNOWN_METRIC_NAME_STRATEGIES.get(cfg.getMetricNameStrategy());
+      if (metricNameStrategy == null) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+            "Unknown metricNameStrategy: " + cfg.getMetricNameStrategy() + " found. Must be one of: " + KNOWN_METRIC_NAME_STRATEGIES.keySet());
+      }
+    }
+    return metricNameStrategy;
+  }
+
+  private InstrumentedHttpListenerFactory.NameStrategy getNameStrategy(UpdateShardHandlerConfig cfg) {
+    InstrumentedHttpListenerFactory.NameStrategy nameStrategy =
+        InstrumentedHttpListenerFactory.KNOWN_METRIC_NAME_STRATEGIES.get(UpdateShardHandlerConfig.DEFAULT_METRICNAMESTRATEGY);
+
+    if (cfg != null)  {
+      nameStrategy = InstrumentedHttpListenerFactory.KNOWN_METRIC_NAME_STRATEGIES.get(cfg.getMetricNameStrategy());
+      if (nameStrategy == null) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+            "Unknown metricNameStrategy: " + cfg.getMetricNameStrategy() + " found. Must be one of: " + KNOWN_METRIC_NAME_STRATEGIES.keySet());
+      }
+    }
+    return nameStrategy;
+  }
+
   @Override
   public String getName() {
     return this.getClass().getName();
@@ -147,8 +166,7 @@ public class UpdateShardHandler implements SolrMetricProducer, SolrInfoBean {
   public void initializeMetrics(SolrMetricManager manager, String registryName, String tag, String scope) {
     registry = manager.registry(registryName);
     String expandedScope = SolrMetricManager.mkName(scope, getCategory().name());
-    //TODO nocommit, replace all the metrics
-    updateOnlyConnectionManager.initializeMetrics(manager, registryName, tag, expandedScope);
+    updateHttpListenerFactory.initializeMetrics(manager, registryName, tag, expandedScope);
     defaultConnectionManager.initializeMetrics(manager, registryName, tag, expandedScope);
     updateExecutor = MetricUtils.instrumentedExecutorService(updateExecutor, this, registry,
         SolrMetricManager.mkName("updateOnlyExecutor", expandedScope, "threadPool"));
@@ -218,7 +236,6 @@ public class UpdateShardHandler implements SolrMetricProducer, SolrInfoBean {
     } finally {
       IOUtils.closeQuietly(updateOnlyClient);
       HttpClientUtil.close(defaultClient);
-      updateOnlyConnectionManager.close();
       defaultConnectionManager.close();
     }
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ab28046f/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpListenerFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpListenerFactory.java b/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpListenerFactory.java
new file mode 100644
index 0000000..e6f1167
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpListenerFactory.java
@@ -0,0 +1,114 @@
+/*
+ * 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.solr.util.stats;
+
+import java.util.HashMap;
+import java.util.Locale;
+import java.util.Map;
+
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.Timer;
+import org.apache.solr.client.solrj.impl.HttpListenerFactory;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.metrics.SolrMetricProducer;
+import org.eclipse.jetty.client.api.Request;
+import org.eclipse.jetty.client.api.Result;
+
+import static org.apache.solr.metrics.SolrMetricManager.mkName;
+
+/**
+ * Sub-class of HttpRequestExecutor which tracks metrics interesting to solr
+ * Inspired and partially copied from dropwizard httpclient library
+ */
+public class InstrumentedHttpListenerFactory implements SolrMetricProducer, HttpListenerFactory {
+
+  public interface NameStrategy {
+    String getNameFor(String scope, Request request);
+  }
+
+  private static final NameStrategy QUERYLESS_URL_AND_METHOD =
+      (scope, request) -> {
+        String schemeHostPort = request.getScheme() + "://" + request.getHost() + ":" + request.getPort() + request.getPath();
+        return mkName(schemeHostPort + "." + methodNameString(request), scope);
+      };
+
+  private static final NameStrategy METHOD_ONLY =
+      (scope, request) -> mkName(methodNameString(request), scope);
+
+  private static final NameStrategy HOST_AND_METHOD =
+      (scope, request) -> {
+        String schemeHostPort = request.getScheme() + "://" + request.getHost() + ":" + request.getPort();
+        return mkName(schemeHostPort + "." + methodNameString(request), scope);
+      };
+
+  public static final Map<String, NameStrategy> KNOWN_METRIC_NAME_STRATEGIES = new HashMap<>(3);
+
+  static  {
+    KNOWN_METRIC_NAME_STRATEGIES.put("queryLessURLAndMethod", QUERYLESS_URL_AND_METHOD);
+    KNOWN_METRIC_NAME_STRATEGIES.put("hostAndMethod", HOST_AND_METHOD);
+    KNOWN_METRIC_NAME_STRATEGIES.put("methodOnly", METHOD_ONLY);
+  }
+
+  protected MetricRegistry metricsRegistry;
+  protected SolrMetricManager metricManager;
+  protected String registryName;
+  protected String scope;
+  protected NameStrategy nameStrategy;
+
+  public InstrumentedHttpListenerFactory(NameStrategy nameStrategy) {
+    this.nameStrategy = nameStrategy;
+  }
+
+  private static String methodNameString(Request request) {
+    return request.getMethod().toLowerCase(Locale.ROOT) + ".requests";
+  }
+
+  @Override
+  public RequestResponseListener get() {
+    return new RequestResponseListener() {
+      Timer.Context timerContext;
+
+      @Override
+      public void onBegin(Request request) {
+        if (metricsRegistry != null) {
+          timerContext = timer(request).time();
+        }
+      }
+
+      @Override
+      public void onComplete(Result result) {
+        if (timerContext != null) {
+          timerContext.stop();
+        }
+      }
+    };
+  }
+
+  private Timer timer(Request request) {
+    return metricsRegistry.timer(nameStrategy.getNameFor(scope, request));
+  }
+
+  @Override
+  public void initializeMetrics(SolrMetricManager manager, String registry, String tag, String scope) {
+    this.metricManager = manager;
+    this.registryName = registry;
+    this.metricsRegistry = manager.registry(registry);
+    this.scope = scope;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ab28046f/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Http2SolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Http2SolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Http2SolrClient.java
index bbe1af41..ad1fcf6 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Http2SolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Http2SolrClient.java
@@ -105,6 +105,7 @@ public class Http2SolrClient extends SolrClient {
 
   private ResponseParser parser = new BinaryResponseParser();
   private volatile RequestWriter requestWriter = new BinaryRequestWriter();
+  private volatile HttpListenerFactory listenerFactory;
 
   private Request.QueuedListener requestQueuedListener = new Request.QueuedListener() {
 
@@ -252,6 +253,11 @@ public class Http2SolrClient extends SolrClient {
                                       OnComplete onComplete,
                                       boolean returnStream) throws IOException, SolrServerException {
     Request req = makeRequest(solrRequest, collection);
+    if (listenerFactory != null) {
+      HttpListenerFactory.RequestResponseListener listener = listenerFactory.get();
+      req.onRequestBegin(listener);
+      req.onComplete(listener);
+    }
 
     if (beginListener != null) {
       // By calling listener here, we will make sure that SolrRequestInfo can be get from the same thread
@@ -325,6 +331,10 @@ public class Http2SolrClient extends SolrClient {
     }
   }
 
+  public void setListenerFactory(HttpListenerFactory listenerFactory) {
+    this.listenerFactory = listenerFactory;
+  }
+
   private Request makeRequest(SolrRequest solrRequest, String collection)
       throws SolrServerException, IOException {
     if (solrRequest.getBasePath() == null && serverBaseUrl == null)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ab28046f/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpListenerFactory.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpListenerFactory.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpListenerFactory.java
new file mode 100644
index 0000000..fb1e00c
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpListenerFactory.java
@@ -0,0 +1,30 @@
+/*
+ * 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.solr.client.solrj.impl;
+
+import org.eclipse.jetty.client.api.Request;
+import org.eclipse.jetty.client.api.Response;
+
+public interface HttpListenerFactory {
+  public interface RequestResponseListener extends Request.BeginListener, Response.CompleteListener {
+
+  }
+
+  public RequestResponseListener get();
+}
+