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

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

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_6x 4449710a3 -> ca12eaa55


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

(cherry picked from commit d2664b100463ada22162d53aad1c6d306d2cc9c1)


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

Branch: refs/heads/branch_6x
Commit: ca12eaa555dc9b6682d19eb637811a4ac6d27751
Parents: 4449710
Author: Shalin Shekhar Mangar <sh...@apache.org>
Authored: Thu Jan 12 17:38:35 2017 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Thu Jan 12 17:38:35 2017 +0530

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  4 +-
 .../org/apache/solr/core/SolrXmlConfig.java     |  9 ++-
 .../component/HttpShardHandlerFactory.java      | 17 ++++-
 .../apache/solr/update/UpdateShardHandler.java  | 13 +++-
 .../solr/update/UpdateShardHandlerConfig.java   | 13 +++-
 .../stats/HttpClientMetricNameStrategy.java     | 28 ++++++++
 .../solr/util/stats/InstrumentedHttpClient.java |  4 +-
 .../stats/InstrumentedHttpRequestExecutor.java  | 76 +++++++++++++++-----
 .../java/org/apache/solr/util/TestHarness.java  |  3 +-
 9 files changed, 140 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ca12eaa5/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 54fa1ce..e95e86f 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -161,7 +161,9 @@ New Features
 
 * SOLR-8530: Add HavingStream to Streaming API and StreamingExpressions (Joel Bernstein)
 
-* 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-9896: Instrument and collect metrics from query, update, core admin and core load thread pools. (shalin)
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ca12eaa5/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 4b960ec..6ccf776 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
@@ -284,6 +284,7 @@ public class SolrXmlConfig {
     int distributedConnectionTimeout = UpdateShardHandlerConfig.DEFAULT_DISTRIBUPDATECONNTIMEOUT;
     int updateConnectionsEvictorSleepDelay = UpdateShardHandlerConfig.DEFAULT_UPDATECONNECTIONSEVICTORSLEEPDELAY;
     int maxUpdateConnectionIdleTime = UpdateShardHandlerConfig.DEFAULT_MAXUPDATECONNECTIONIDLETIME;
+    String metricNameStrategy = UpdateShardHandlerConfig.DEFAULT_METRICNAMESTRATEGY;
 
     Object muc = nl.remove("maxUpdateConnections");
     if (muc != null) {
@@ -321,11 +322,17 @@ 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, updateConnectionsEvictorSleepDelay, maxUpdateConnectionIdleTime);
+        distributedConnectionTimeout, updateConnectionsEvictorSleepDelay, maxUpdateConnectionIdleTime, metricNameStrategy);
 
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ca12eaa5/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 a22c8f7..f015590 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
@@ -43,6 +43,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;
@@ -61,12 +62,15 @@ import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.update.UpdateShardHandler;
 import org.apache.solr.update.UpdateShardHandlerConfig;
 import org.apache.solr.util.DefaultSolrThreadFactory;
-import org.apache.solr.util.stats.InstrumentedHttpClient;
+import org.apache.solr.util.stats.HttpClientMetricNameStrategy;
 import org.apache.solr.util.stats.InstrumentedPoolingClientConnectionManager;
+import org.apache.solr.util.stats.InstrumentedHttpClient;
 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;
+
 public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.apache.solr.util.plugin.PluginInfoInitialized, SolrMetricProducer {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private static final String DEFAULT_SCHEME = "http";
@@ -106,6 +110,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);
@@ -161,6 +167,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);
@@ -197,7 +210,7 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
     clientConnectionManager = new InstrumentedPoolingClientConnectionManager(SchemeRegistryFactory.createSystemDefault());
     clientConnectionManager.setDefaultMaxPerRoute(maxConnectionsPerHost);
     clientConnectionManager.setMaxTotal(maxConnections);
-    InstrumentedHttpClient httpClient = new InstrumentedHttpClient(clientConnectionManager);
+    InstrumentedHttpClient httpClient = new InstrumentedHttpClient(clientConnectionManager, metricNameStrategy);
     HttpClientUtil.configureClient(httpClient, clientParams);
     this.defaultClient = httpClient;
     this.idleConnectionsEvictor = new UpdateShardHandler.IdleConnectionsEvictor(clientConnectionManager,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ca12eaa5/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 5f09ad2..234d101 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
@@ -41,11 +41,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.InstrumentedHttpClient;
 import org.apache.solr.util.stats.InstrumentedPoolingClientConnectionManager;
 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());
@@ -81,7 +84,15 @@ public class UpdateShardHandler implements SolrMetricProducer, SolrInfoMBean {
 
     ModifiableSolrParams clientParams = getClientParams();
     log.info("Creating UpdateShardHandler HTTP client with params: {}", clientParams);
-    InstrumentedHttpClient httpClient = new InstrumentedHttpClient(clientConnectionManager);
+    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());
+      }
+    }
+    InstrumentedHttpClient httpClient = new InstrumentedHttpClient(clientConnectionManager, metricNameStrategy);
     HttpClientUtil.configureClient(httpClient, clientParams);
     client = httpClient;
     if (cfg != null)  {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ca12eaa5/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 915b878..180c0c1 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateShardHandlerConfig.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateShardHandlerConfig.java
@@ -24,11 +24,13 @@ public class UpdateShardHandlerConfig {
   public static final int DEFAULT_MAXUPDATECONNECTIONSPERHOST = 100000;
   public static final int DEFAULT_UPDATECONNECTIONSEVICTORSLEEPDELAY = 5000;
   public static final int DEFAULT_MAXUPDATECONNECTIONIDLETIME = 40000;
+  public static final String DEFAULT_METRICNAMESTRATEGY = "queryLessURLAndMethod";
 
   public static final UpdateShardHandlerConfig DEFAULT
       = new UpdateShardHandlerConfig(DEFAULT_MAXUPDATECONNECTIONS, DEFAULT_MAXUPDATECONNECTIONSPERHOST,
                                      DEFAULT_DISTRIBUPDATESOTIMEOUT, DEFAULT_DISTRIBUPDATECONNTIMEOUT,
-                                      DEFAULT_UPDATECONNECTIONSEVICTORSLEEPDELAY, DEFAULT_MAXUPDATECONNECTIONIDLETIME);
+                                      DEFAULT_UPDATECONNECTIONSEVICTORSLEEPDELAY, DEFAULT_MAXUPDATECONNECTIONIDLETIME,
+                                      DEFAULT_METRICNAMESTRATEGY);
 
   private final int maxUpdateConnections;
 
@@ -38,15 +40,18 @@ public class UpdateShardHandlerConfig {
 
   private final int distributedConnectionTimeout;
 
+  private final String metricNameStrategy;
+
   private final int updateConnectionsEvictorSleepDelay;
 
   private final int maxUpdateConnectionIdleTime;
 
-  public UpdateShardHandlerConfig(int maxUpdateConnections, int maxUpdateConnectionsPerHost, int distributedSocketTimeout, int distributedConnectionTimeout, int updateConnectionsEvictorSleepDelay, int maxUpdateConnectionIdleTime) {
+  public UpdateShardHandlerConfig(int maxUpdateConnections, int maxUpdateConnectionsPerHost, int distributedSocketTimeout, int distributedConnectionTimeout, int updateConnectionsEvictorSleepDelay, int maxUpdateConnectionIdleTime, String metricNameStrategy) {
     this.maxUpdateConnections = maxUpdateConnections;
     this.maxUpdateConnectionsPerHost = maxUpdateConnectionsPerHost;
     this.distributedSocketTimeout = distributedSocketTimeout;
     this.distributedConnectionTimeout = distributedConnectionTimeout;
+    this.metricNameStrategy = metricNameStrategy;
     this.updateConnectionsEvictorSleepDelay = updateConnectionsEvictorSleepDelay;
     this.maxUpdateConnectionIdleTime = maxUpdateConnectionIdleTime;
   }
@@ -67,6 +72,10 @@ public class UpdateShardHandlerConfig {
     return distributedConnectionTimeout;
   }
 
+  public String getMetricNameStrategy() {
+    return metricNameStrategy;
+  }
+
   public int getUpdateConnectionsEvictorSleepDelay() {
     return updateConnectionsEvictorSleepDelay;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ca12eaa5/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/ca12eaa5/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpClient.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpClient.java b/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpClient.java
index 16306ac..9b7f167 100644
--- a/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpClient.java
+++ b/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpClient.java
@@ -27,9 +27,9 @@ public class InstrumentedHttpClient extends DefaultHttpClient implements SolrMet
 
   protected final InstrumentedHttpRequestExecutor requestExecutor;
 
-  public InstrumentedHttpClient(ClientConnectionManager conman) {
+  public InstrumentedHttpClient(ClientConnectionManager conman, HttpClientMetricNameStrategy metricNameStrategy) {
     super(conman);
-    this.requestExecutor = new InstrumentedHttpRequestExecutor();
+    this.requestExecutor = new InstrumentedHttpRequestExecutor(metricNameStrategy);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ca12eaa5/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 54a6e73..3f2ec9a 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
@@ -18,8 +18,11 @@
 package org.apache.solr.util.stats;
 
 import java.io.IOException;
+import java.net.URI;
 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 +38,67 @@ 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 {
+            RequestLine requestLine = request.getRequestLine();
+            if (request instanceof EntityEnclosingRequestWrapper) {
+              EntityEnclosingRequestWrapper wrapper = (EntityEnclosingRequestWrapper) request;
+              if (wrapper.getOriginal() != null)  {
+                requestLine = wrapper.getOriginal().getRequestLine();
+              }
+            }
+            final URIBuilder url = new URIBuilder(requestLine.getUri());
+            return SolrMetricManager.mkName(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) -> {
+        RequestLine requestLine = request.getRequestLine();
+        if (request instanceof EntityEnclosingRequestWrapper) {
+          EntityEnclosingRequestWrapper wrapper = (EntityEnclosingRequestWrapper) request;
+          if (wrapper.getOriginal() != null)  {
+            requestLine = wrapper.getOriginal().getRequestLine();
+          }
+        }
+        final URI uri = URI.create(requestLine.getUri());
+        String schemeHostPort = uri.getScheme() + "://" + uri.getHost() + ":" + uri.getPort();
+        return mkName(schemeHostPort + "." + methodNameString(request), scope);
+      };
+
+  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 +107,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 +120,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,19 +129,4 @@ public class InstrumentedHttpRequestExecutor extends HttpRequestExecutor impleme
     this.scope = scope;
   }
 
-  private String getNameFor(HttpRequest request) {
-    try {
-      RequestLine requestLine = request.getRequestLine();
-      if (request instanceof EntityEnclosingRequestWrapper) {
-        EntityEnclosingRequestWrapper wrapper = (EntityEnclosingRequestWrapper) request;
-        if (wrapper.getOriginal() != null)  {
-          requestLine = wrapper.getOriginal().getRequestLine();
-        }
-      }
-      final URIBuilder url = new URIBuilder(requestLine.getUri());
-      return SolrMetricManager.mkName(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/ca12eaa5/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 030d8a5..80760c6 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, 5000, 50000);
+                                       30000, 30000, 5000, 50000,
+        UpdateShardHandlerConfig.DEFAULT_METRICNAMESTRATEGY);
     // universal default metric reporter
     Map<String,String> attributes = new HashMap<>();
     attributes.put("name", "default");