You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by kr...@apache.org on 2017/01/12 16:51:58 UTC

[41/43] lucene-solr:jira/solr-8593: SOLR-9948: Add a way to configure granularity of metrics for http connections

SOLR-9948: Add a way to configure granularity of metrics for http connections


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

Branch: refs/heads/jira/solr-8593
Commit: d2664b100463ada22162d53aad1c6d306d2cc9c1
Parents: 7435ab1
Author: Shalin Shekhar Mangar <sh...@apache.org>
Authored: Thu Jan 12 13:00:00 2017 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Thu Jan 12 13:00:00 2017 +0530

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  4 +-
 .../org/apache/solr/core/SolrXmlConfig.java     |  9 ++-
 .../component/HttpShardHandlerFactory.java      | 15 +++-
 .../apache/solr/update/UpdateShardHandler.java  | 14 +++-
 .../solr/update/UpdateShardHandlerConfig.java   | 14 +++-
 .../stats/HttpClientMetricNameStrategy.java     | 28 +++++++
 .../stats/InstrumentedHttpRequestExecutor.java  | 81 +++++++++++++++-----
 .../java/org/apache/solr/util/TestHarness.java  |  3 +-
 8 files changed, 143 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d2664b10/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index afcd10b..0cf50d4 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -209,7 +209,9 @@ New Features
 
 * SOLR-9725: Substitute properties into JdbcDataSource configuration ( Jamie Jackson, Yuri Sashevsky via Mikhail Khludnev)
 
-* SOLR-9877: SOLR-9923: Use instrumented http client and connection pool. (shalin)
+* SOLR-9877: SOLR-9923: SOLR-9948: Use instrumented http client and connection pool in HttpShardHandler and
+  UpdateShardHandler. The metrics are collected per query-less URL and method by default but it can be configured
+  to host/method and per-method as well. (shalin)
 
 * SOLR-9880: Add Ganglia, Graphite and SLF4J metrics reporters. (ab)
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d2664b10/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java b/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
index 49d9ae5..e41cd8d 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
@@ -282,6 +282,7 @@ public class SolrXmlConfig {
     int maxUpdateConnectionsPerHost = UpdateShardHandlerConfig.DEFAULT_MAXUPDATECONNECTIONSPERHOST;
     int distributedSocketTimeout = UpdateShardHandlerConfig.DEFAULT_DISTRIBUPDATESOTIMEOUT;
     int distributedConnectionTimeout = UpdateShardHandlerConfig.DEFAULT_DISTRIBUPDATECONNTIMEOUT;
+    String metricNameStrategy = UpdateShardHandlerConfig.DEFAULT_METRICNAMESTRATEGY;
 
     Object muc = nl.remove("maxUpdateConnections");
     if (muc != null) {
@@ -307,10 +308,16 @@ public class SolrXmlConfig {
       defined = true;
     }
 
+    Object mns = nl.remove("metricNameStrategy");
+    if (mns != null)  {
+      metricNameStrategy = mns.toString();
+      defined = true;
+    }
+
     if (!defined && !alwaysDefine)
       return null;
 
-    return new UpdateShardHandlerConfig(maxUpdateConnections, maxUpdateConnectionsPerHost, distributedSocketTimeout, distributedConnectionTimeout);
+    return new UpdateShardHandlerConfig(maxUpdateConnections, maxUpdateConnectionsPerHost, distributedSocketTimeout, distributedConnectionTimeout, metricNameStrategy);
 
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d2664b10/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 258be97..be6e763 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
@@ -25,6 +25,7 @@ import org.apache.solr.client.solrj.impl.LBHttpSolrClient;
 import org.apache.solr.client.solrj.impl.LBHttpSolrClient.Builder;
 import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.cloud.ZkController;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
@@ -41,6 +42,7 @@ import org.apache.solr.metrics.SolrMetricProducer;
 import org.apache.solr.update.UpdateShardHandlerConfig;
 import org.apache.solr.request.SolrQueryRequest;
 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.InstrumentedPoolingHttpClientConnectionManager;
 import org.apache.solr.util.stats.MetricUtils;
@@ -61,6 +63,8 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.TimeUnit;
 
+import static org.apache.solr.util.stats.InstrumentedHttpRequestExecutor.KNOWN_METRIC_NAME_STRATEGIES;
+
 
 public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.apache.solr.util.plugin.PluginInfoInitialized, SolrMetricProducer {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@@ -97,6 +101,8 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
 
   private String scheme = null;
 
+  private HttpClientMetricNameStrategy metricNameStrategy;
+
   private final Random r = new Random();
 
   private final ReplicaListTransformer shufflingReplicaListTransformer = new ShufflingReplicaListTransformer(r);
@@ -148,6 +154,13 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
       this.scheme = StringUtils.removeEnd(this.scheme, "://");
     }
 
+    String strategy = getParameter(args, "metricNameStrategy", UpdateShardHandlerConfig.DEFAULT_METRICNAMESTRATEGY, sb);
+    this.metricNameStrategy = KNOWN_METRIC_NAME_STRATEGIES.get(strategy);
+    if (this.metricNameStrategy == null)  {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+          "Unknown metricNameStrategy: " + strategy + " found. Must be one of: " + KNOWN_METRIC_NAME_STRATEGIES.keySet());
+    }
+
     this.connectionTimeout = getParameter(args, HttpClientUtil.PROP_CONNECTION_TIMEOUT, connectionTimeout, sb);
     this.maxConnectionsPerHost = getParameter(args, HttpClientUtil.PROP_MAX_CONNECTIONS_PER_HOST, maxConnectionsPerHost,sb);
     this.maxConnections = getParameter(args, HttpClientUtil.PROP_MAX_CONNECTIONS, maxConnections,sb);
@@ -177,7 +190,7 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
     );
 
     ModifiableSolrParams clientParams = getClientParams();
-    httpRequestExecutor = new InstrumentedHttpRequestExecutor();
+    httpRequestExecutor = new InstrumentedHttpRequestExecutor(this.metricNameStrategy);
     clientConnectionManager = new InstrumentedPoolingHttpClientConnectionManager(HttpClientUtil.getSchemaRegisteryProvider().getSchemaRegistry());
     this.defaultClient = HttpClientUtil.createClient(clientParams, clientConnectionManager, false, httpRequestExecutor);
     this.loadbalancer = createLoadbalancer(defaultClient);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d2664b10/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 f13cfb5..67447a3 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
@@ -34,11 +34,14 @@ import org.apache.solr.common.util.SolrjNamedThreadFactory;
 import org.apache.solr.core.SolrInfoMBean;
 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.InstrumentedHttpRequestExecutor;
 import org.apache.solr.util.stats.InstrumentedPoolingHttpClientConnectionManager;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.solr.util.stats.InstrumentedHttpRequestExecutor.KNOWN_METRIC_NAME_STRATEGIES;
+
 public class UpdateShardHandler implements SolrMetricProducer, SolrInfoMBean {
   
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@@ -74,7 +77,16 @@ public class UpdateShardHandler implements SolrMetricProducer, SolrInfoMBean {
       clientParams.set(HttpClientUtil.PROP_SO_TIMEOUT, cfg.getDistributedSocketTimeout());
       clientParams.set(HttpClientUtil.PROP_CONNECTION_TIMEOUT, cfg.getDistributedConnectionTimeout());
     }
-    httpRequestExecutor = new InstrumentedHttpRequestExecutor();
+    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(metricNameStrategy);
     client = HttpClientUtil.createClient(clientParams, clientConnectionManager, false, httpRequestExecutor);
 
     // following is done only for logging complete configuration.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d2664b10/solr/core/src/java/org/apache/solr/update/UpdateShardHandlerConfig.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/UpdateShardHandlerConfig.java b/solr/core/src/java/org/apache/solr/update/UpdateShardHandlerConfig.java
index ffb06c4..d31ce50 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateShardHandlerConfig.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateShardHandlerConfig.java
@@ -22,10 +22,12 @@ public class UpdateShardHandlerConfig {
   public static final int DEFAULT_DISTRIBUPDATESOTIMEOUT = 600000;
   public static final int DEFAULT_MAXUPDATECONNECTIONS = 100000;
   public static final int DEFAULT_MAXUPDATECONNECTIONSPERHOST = 100000;
+  public static final String DEFAULT_METRICNAMESTRATEGY = "queryLessURLAndMethod";
 
   public static final UpdateShardHandlerConfig DEFAULT
       = new UpdateShardHandlerConfig(DEFAULT_MAXUPDATECONNECTIONS, DEFAULT_MAXUPDATECONNECTIONSPERHOST,
-                                     DEFAULT_DISTRIBUPDATESOTIMEOUT, DEFAULT_DISTRIBUPDATECONNTIMEOUT);
+                                     DEFAULT_DISTRIBUPDATESOTIMEOUT, DEFAULT_DISTRIBUPDATECONNTIMEOUT,
+                                      DEFAULT_METRICNAMESTRATEGY);
 
   private final int maxUpdateConnections;
 
@@ -35,11 +37,15 @@ public class UpdateShardHandlerConfig {
 
   private final int distributedConnectionTimeout;
 
-  public UpdateShardHandlerConfig(int maxUpdateConnections, int maxUpdateConnectionsPerHost, int distributedSocketTimeout, int distributedConnectionTimeout) {
+  private final String metricNameStrategy;
+
+  public UpdateShardHandlerConfig(int maxUpdateConnections, int maxUpdateConnectionsPerHost, int distributedSocketTimeout, int distributedConnectionTimeout,
+                                  String metricNameStrategy) {
     this.maxUpdateConnections = maxUpdateConnections;
     this.maxUpdateConnectionsPerHost = maxUpdateConnectionsPerHost;
     this.distributedSocketTimeout = distributedSocketTimeout;
     this.distributedConnectionTimeout = distributedConnectionTimeout;
+    this.metricNameStrategy = metricNameStrategy;
   }
 
   public int getMaxUpdateConnectionsPerHost() {
@@ -57,4 +63,8 @@ public class UpdateShardHandlerConfig {
   public int getDistributedConnectionTimeout() {
     return distributedConnectionTimeout;
   }
+
+  public String getMetricNameStrategy() {
+    return metricNameStrategy;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d2664b10/solr/core/src/java/org/apache/solr/util/stats/HttpClientMetricNameStrategy.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/stats/HttpClientMetricNameStrategy.java b/solr/core/src/java/org/apache/solr/util/stats/HttpClientMetricNameStrategy.java
new file mode 100644
index 0000000..930c5f7
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/util/stats/HttpClientMetricNameStrategy.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.solr.util.stats;
+
+import org.apache.http.HttpRequest;
+
+/**
+ * Strategy for creating metric names for HttpClient
+ * Copied from metrics-httpclient library
+ */
+public interface HttpClientMetricNameStrategy {
+  String getNameFor(String scope, HttpRequest request);
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d2664b10/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpRequestExecutor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpRequestExecutor.java b/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpRequestExecutor.java
index 0caa2d1..bd8d368 100644
--- a/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpRequestExecutor.java
+++ b/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpRequestExecutor.java
@@ -19,7 +19,9 @@ package org.apache.solr.util.stats;
 
 import java.io.IOException;
 import java.net.URISyntaxException;
+import java.util.HashMap;
 import java.util.Locale;
+import java.util.Map;
 
 import com.codahale.metrics.MetricRegistry;
 import com.codahale.metrics.Timer;
@@ -35,13 +37,72 @@ import org.apache.http.protocol.HttpRequestExecutor;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.metrics.SolrMetricProducer;
 
+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 InstrumentedHttpRequestExecutor extends HttpRequestExecutor implements SolrMetricProducer {
+  public static final HttpClientMetricNameStrategy QUERYLESS_URL_AND_METHOD =
+      (scope, request) -> {
+        try {
+          final RequestLine requestLine = request.getRequestLine();
+          String schemeHostPort = null;
+          if (request instanceof HttpRequestWrapper) {
+            HttpRequestWrapper wrapper = (HttpRequestWrapper) request;
+            if (wrapper.getTarget() != null) {
+              schemeHostPort = wrapper.getTarget().getSchemeName() + "://" + wrapper.getTarget().getHostName() + ":" + wrapper.getTarget().getPort();
+            }
+          }
+          final URIBuilder url = new URIBuilder(requestLine.getUri());
+          return mkName((schemeHostPort != null ? schemeHostPort : "") + url.removeQuery().build().toString() + "." + methodNameString(request), scope);
+        } catch (URISyntaxException e) {
+          throw new IllegalArgumentException(e);
+        }
+      };
+
+  public static final HttpClientMetricNameStrategy METHOD_ONLY =
+      (scope, request) -> mkName(methodNameString(request), scope);
+
+  public static final HttpClientMetricNameStrategy HOST_AND_METHOD =
+      (scope, request) -> {
+        try {
+          final RequestLine requestLine = request.getRequestLine();
+          String schemeHostPort = null;
+          if (request instanceof HttpRequestWrapper) {
+            HttpRequestWrapper wrapper = (HttpRequestWrapper) request;
+            if (wrapper.getTarget() != null) {
+              schemeHostPort = wrapper.getTarget().getSchemeName() + "://" + wrapper.getTarget().getHostName() + ":" + wrapper.getTarget().getPort();
+            }
+          }
+          final URIBuilder url = new URIBuilder(requestLine.getUri());
+          return mkName((schemeHostPort != null ? schemeHostPort : "") + "." + methodNameString(request), scope);
+        } catch (URISyntaxException e) {
+          throw new IllegalArgumentException(e);
+        }
+      };
+
+  public static final Map<String, HttpClientMetricNameStrategy> 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 String scope;
+  protected HttpClientMetricNameStrategy nameStrategy;
+
+  public InstrumentedHttpRequestExecutor(int waitForContinue, HttpClientMetricNameStrategy nameStrategy) {
+    super(waitForContinue);
+    this.nameStrategy = nameStrategy;
+  }
+
+  public InstrumentedHttpRequestExecutor(HttpClientMetricNameStrategy nameStrategy) {
+    this.nameStrategy = nameStrategy;
+  }
 
   private static String methodNameString(HttpRequest request) {
     return request.getRequestLine().getMethod().toLowerCase(Locale.ROOT) + ".requests";
@@ -50,7 +111,7 @@ public class InstrumentedHttpRequestExecutor extends HttpRequestExecutor impleme
   @Override
   public HttpResponse execute(HttpRequest request, HttpClientConnection conn, HttpContext context) throws IOException, HttpException {
     Timer.Context timerContext = null;
-    if (metricsRegistry != null)  {
+    if (metricsRegistry != null) {
       timerContext = timer(request).time();
     }
     try {
@@ -63,7 +124,7 @@ public class InstrumentedHttpRequestExecutor extends HttpRequestExecutor impleme
   }
 
   private Timer timer(HttpRequest request) {
-    return metricsRegistry.timer(getNameFor(request));
+    return metricsRegistry.timer(nameStrategy.getNameFor(scope, request));
   }
 
   @Override
@@ -72,20 +133,4 @@ public class InstrumentedHttpRequestExecutor extends HttpRequestExecutor impleme
     this.scope = scope;
   }
 
-  private String getNameFor(HttpRequest request) {
-    try {
-      final RequestLine requestLine = request.getRequestLine();
-      String schemeHostPort = null;
-      if (request instanceof HttpRequestWrapper) {
-        HttpRequestWrapper wrapper = (HttpRequestWrapper) request;
-        if (wrapper.getTarget() != null)  {
-          schemeHostPort = wrapper.getTarget().getSchemeName() + "://" + wrapper.getTarget().getHostName() + ":" +  wrapper.getTarget().getPort();
-        }
-      }
-      final URIBuilder url = new URIBuilder(requestLine.getUri());
-      return SolrMetricManager.mkName((schemeHostPort != null ? schemeHostPort : "") + url.removeQuery().build().toString() + "." + methodNameString(request), scope);
-    } catch (URISyntaxException e) {
-      throw new IllegalArgumentException(e);
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d2664b10/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java b/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
index be8a24c..2386681 100644
--- a/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
+++ b/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
@@ -190,7 +190,8 @@ public class TestHarness extends BaseTestHarness {
     UpdateShardHandlerConfig updateShardHandlerConfig
         = new UpdateShardHandlerConfig(UpdateShardHandlerConfig.DEFAULT_MAXUPDATECONNECTIONS,
                                        UpdateShardHandlerConfig.DEFAULT_MAXUPDATECONNECTIONSPERHOST,
-                                       30000, 30000);
+                                       30000, 30000,
+                                        UpdateShardHandlerConfig.DEFAULT_METRICNAMESTRATEGY);
     // universal default metric reporter
     Map<String,String> attributes = new HashMap<>();
     attributes.put("name", "default");