You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2016/12/20 10:48:23 UTC

[18/44] lucene-solr:jira/solr-9854: SOLR-9812: Added a new /admin/metrics API to return all metrics collected by Solr via API

SOLR-9812: Added a new /admin/metrics API to return all metrics collected by Solr via API


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

Branch: refs/heads/jira/solr-9854
Commit: aa9b02bb16afe2af8c2437ffab46f4a09bda684e
Parents: 5f0637c
Author: Shalin Shekhar Mangar <sh...@apache.org>
Authored: Fri Dec 16 21:36:01 2016 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Fri Dec 16 21:36:01 2016 +0530

----------------------------------------------------------------------
 lucene/ivy-versions.properties                  |   2 -
 .../plugin/AnalyticsStatisticsCollector.java    |   4 +-
 .../apache/solr/cloud/OverseerStatusCmd.java    |   4 +-
 .../org/apache/solr/core/CoreContainer.java     |   7 +-
 .../apache/solr/handler/RequestHandlerBase.java |   4 +-
 .../solr/handler/admin/MetricsHandler.java      | 161 +++++++++++++++++++
 .../solr/metrics/SolrCoreMetricManager.java     |   6 -
 .../apache/solr/metrics/SolrMetricReporter.java |   1 -
 .../solr/security/PermissionNameProvider.java   |   1 +
 .../apache/solr/servlet/SolrDispatchFilter.java |   2 -
 .../SolrMetricsServletContextListener.java      |  29 ----
 .../org/apache/solr/util/stats/MetricUtils.java | 144 +++++++++++++++++
 .../org/apache/solr/util/stats/TimerUtils.java  |  58 -------
 .../solr/handler/admin/MetricsHandlerTest.java  |  95 +++++++++++
 .../apache/solr/util/stats/MetricUtilsTest.java |  58 +++++++
 .../apache/solr/util/stats/TimerUtilsTest.java  |  58 -------
 solr/licenses/metrics-json-3.1.2.jar.sha1       |   1 -
 solr/licenses/metrics-servlets-3.1.2.jar.sha1   |   1 -
 solr/server/ivy.xml                             |   5 -
 .../apache/solr/common/params/CommonParams.java |   4 +-
 solr/webapp/web/WEB-INF/web.xml                 |  17 --
 21 files changed, 473 insertions(+), 189 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aa9b02bb/lucene/ivy-versions.properties
----------------------------------------------------------------------
diff --git a/lucene/ivy-versions.properties b/lucene/ivy-versions.properties
index 082c303..2f44f7e 100644
--- a/lucene/ivy-versions.properties
+++ b/lucene/ivy-versions.properties
@@ -77,8 +77,6 @@ io.dropwizard.metrics.version = 3.1.2
 /io.dropwizard.metrics/metrics-core = ${io.dropwizard.metrics.version}
 /io.dropwizard.metrics/metrics-healthchecks = ${io.dropwizard.metrics.version}
 /io.dropwizard.metrics/metrics-jetty9 = ${io.dropwizard.metrics.version}
-/io.dropwizard.metrics/metrics-json = ${io.dropwizard.metrics.version}
-/io.dropwizard.metrics/metrics-servlets = ${io.dropwizard.metrics.version}
 /io.dropwizard.metrics/metrics-jvm = ${io.dropwizard.metrics.version}
 
 io.netty.netty-all.version = 4.0.36.Final

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aa9b02bb/solr/contrib/analytics/src/java/org/apache/solr/analytics/plugin/AnalyticsStatisticsCollector.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/plugin/AnalyticsStatisticsCollector.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/plugin/AnalyticsStatisticsCollector.java
index e64c950..b22dcb5 100644
--- a/solr/contrib/analytics/src/java/org/apache/solr/analytics/plugin/AnalyticsStatisticsCollector.java
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/plugin/AnalyticsStatisticsCollector.java
@@ -21,7 +21,7 @@ import java.util.concurrent.atomic.AtomicLong;
 import com.codahale.metrics.Timer;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
-import org.apache.solr.util.stats.TimerUtils;
+import org.apache.solr.util.stats.MetricUtils;
 
 public class AnalyticsStatisticsCollector {
   private final AtomicLong numRequests;
@@ -95,7 +95,7 @@ public class AnalyticsStatisticsCollector {
     lst.add("rangeFacets", numRangeFacets.longValue());
     lst.add("queryFacets", numQueryFacets.longValue());
     lst.add("queriesInQueryFacets", numQueries.longValue());
-    TimerUtils.addMetrics(lst, requestTimes);
+    MetricUtils.addMetrics(lst, requestTimes);
     return lst;
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aa9b02bb/solr/core/src/java/org/apache/solr/cloud/OverseerStatusCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerStatusCmd.java b/solr/core/src/java/org/apache/solr/cloud/OverseerStatusCmd.java
index 901a884..a24deb1 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerStatusCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerStatusCmd.java
@@ -30,7 +30,7 @@ import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
-import org.apache.solr.util.stats.TimerUtils;
+import org.apache.solr.util.stats.MetricUtils;
 import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -100,7 +100,7 @@ public class OverseerStatusCmd implements Cmd {
         lst.add("errors", errors);
       }
       Timer timer = entry.getValue().requestTime;
-      TimerUtils.addMetrics(lst, timer);
+      MetricUtils.addMetrics(lst, timer);
     }
     results.add("overseer_operations", overseerStats);
     results.add("collection_operations", collectionStats);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aa9b02bb/solr/core/src/java/org/apache/solr/core/CoreContainer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index 1e29525..c423985 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -33,7 +33,6 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
 
-import com.codahale.metrics.MetricRegistry;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
 import org.apache.http.auth.AuthSchemeProvider;
@@ -59,6 +58,7 @@ import org.apache.solr.handler.admin.CollectionsHandler;
 import org.apache.solr.handler.admin.ConfigSetsHandler;
 import org.apache.solr.handler.admin.CoreAdminHandler;
 import org.apache.solr.handler.admin.InfoHandler;
+import org.apache.solr.handler.admin.MetricsHandler;
 import org.apache.solr.handler.admin.SecurityConfHandler;
 import org.apache.solr.handler.admin.SecurityConfHandlerLocal;
 import org.apache.solr.handler.admin.SecurityConfHandlerZk;
@@ -66,7 +66,6 @@ import org.apache.solr.handler.admin.ZookeeperInfoHandler;
 import org.apache.solr.handler.component.ShardHandlerFactory;
 import org.apache.solr.logging.LogWatcher;
 import org.apache.solr.logging.MDCLoggingContext;
-import org.apache.solr.metrics.SolrCoreMetricManager;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.metrics.SolrMetricProducer;
 import org.apache.solr.request.SolrRequestHandler;
@@ -89,6 +88,7 @@ import static org.apache.solr.common.params.CommonParams.COLLECTIONS_HANDLER_PAT
 import static org.apache.solr.common.params.CommonParams.CONFIGSETS_HANDLER_PATH;
 import static org.apache.solr.common.params.CommonParams.CORES_HANDLER_PATH;
 import static org.apache.solr.common.params.CommonParams.INFO_HANDLER_PATH;
+import static org.apache.solr.common.params.CommonParams.METRICS_PATH;
 import static org.apache.solr.common.params.CommonParams.ZK_PATH;
 import static org.apache.solr.security.AuthenticationPlugin.AUTHENTICATION_PLUGIN_PROP;
 
@@ -160,6 +160,8 @@ public class CoreContainer {
 
   private BackupRepositoryFactory backupRepoFactory;
 
+  protected MetricsHandler metricsHandler;
+
   /**
    * This method instantiates a new instance of {@linkplain BackupRepository}.
    *
@@ -476,6 +478,7 @@ public class CoreContainer {
     infoHandler        = createHandler(INFO_HANDLER_PATH, cfg.getInfoHandlerClass(), InfoHandler.class);
     coreAdminHandler   = createHandler(CORES_HANDLER_PATH, cfg.getCoreAdminHandlerClass(), CoreAdminHandler.class);
     configSetsHandler = createHandler(CONFIGSETS_HANDLER_PATH, cfg.getConfigSetsHandlerClass(), ConfigSetsHandler.class);
+    metricsHandler = createHandler(METRICS_PATH, MetricsHandler.class.getName(), MetricsHandler.class);
     containerHandlers.put(AUTHZ_PATH, securityConfHandler);
     securityConfHandler.initializeMetrics(SolrInfoMBean.Group.node.toString(), AUTHZ_PATH);
     containerHandlers.put(AUTHC_PATH, securityConfHandler);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aa9b02bb/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java b/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
index 7310c32..7d9d464 100644
--- a/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
+++ b/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
@@ -39,7 +39,7 @@ import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.search.SyntaxError;
 import org.apache.solr.util.SolrPluginUtils;
-import org.apache.solr.util.stats.TimerUtils;
+import org.apache.solr.util.stats.MetricUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -290,7 +290,7 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
     lst.add("serverErrors", numServerErrors.getCount());
     lst.add("clientErrors", numClientErrors.getCount());
     lst.add("timeouts", numTimeouts.getCount());
-    TimerUtils.addMetrics(lst, requestTimes);
+    MetricUtils.addMetrics(lst, requestTimes);
     return lst;
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aa9b02bb/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java
new file mode 100644
index 0000000..1adc480
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java
@@ -0,0 +1,161 @@
+/*
+ * 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.handler.admin;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.Meter;
+import com.codahale.metrics.MetricFilter;
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.Timer;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.handler.RequestHandlerBase;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.security.AuthorizationContext;
+import org.apache.solr.security.PermissionNameProvider;
+import org.apache.solr.util.stats.MetricUtils;
+
+/**
+ * Request handler to return metrics
+ */
+public class MetricsHandler extends RequestHandlerBase implements PermissionNameProvider {
+  final CoreContainer container;
+
+  public MetricsHandler() {
+    this.container = null;
+  }
+
+  public MetricsHandler(CoreContainer container) {
+    this.container = container;
+  }
+
+  @Override
+  public Name getPermissionName(AuthorizationContext request) {
+    return Name.METRICS_READ_PERM;
+  }
+
+  @Override
+  public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
+    if (container == null) {
+      throw new SolrException(SolrException.ErrorCode.INVALID_STATE, "Core container instance not initialized");
+    }
+
+    List<MetricType> metricTypes = parseMetricTypes(req);
+    List<MetricFilter> metricFilters = metricTypes.stream().map(MetricType::asMetricFilter).collect(Collectors.toList());
+    List<Group> requestedGroups = parseGroups(req);
+
+    NamedList response = new NamedList();
+    for (Group group : requestedGroups) {
+      String registryName = SolrMetricManager.getRegistryName(group);
+      if (group == Group.core) {
+        // this requires special handling because of the way we create registry name for a core (deeply nested)
+        container.getAllCoreNames().forEach(s -> {
+          String coreRegistryName;
+          try (SolrCore core = container.getCore(s)) {
+            coreRegistryName = core.getMetricManager().getRegistryName();
+          }
+          MetricRegistry registry = SolrMetricManager.registry(coreRegistryName);
+          response.add(coreRegistryName, MetricUtils.toNamedList(registry, metricFilters));
+        });
+      } else {
+        MetricRegistry registry = SolrMetricManager.registry(registryName);
+        response.add(registryName, MetricUtils.toNamedList(registry, metricFilters));
+      }
+    }
+    rsp.getValues().add("metrics", response);
+  }
+
+  private List<Group> parseGroups(SolrQueryRequest req) {
+    String[] groupStr = req.getParams().getParams("group");
+    List<String> groups = Collections.emptyList();
+    if (groupStr != null && groupStr.length > 0) {
+      groups = new ArrayList<>();
+      for (String g : groupStr) {
+        groups.addAll(StrUtils.splitSmart(g, ','));
+      }
+    }
+
+    List<Group> requestedGroups = Arrays.asList(Group.values()); // by default we return all groups
+    try {
+      if (groups.size() > 0 && !groups.contains("all")) {
+        requestedGroups = groups.stream().map(String::trim).map(Group::valueOf).collect(Collectors.toList());
+      }
+    } catch (IllegalArgumentException e) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Invalid group in: " + groups + " specified. Must be one of (all, jvm, jetty, http, node, core)", e);
+    }
+    return requestedGroups;
+  }
+
+  private List<MetricType> parseMetricTypes(SolrQueryRequest req) {
+    String[] typeStr = req.getParams().getParams("type");
+    List<String> types = Collections.emptyList();
+    if (typeStr != null && typeStr.length > 0)  {
+      types = new ArrayList<>();
+      for (String type : typeStr) {
+        types.addAll(StrUtils.splitSmart(type, ','));
+      }
+    }
+
+    List<MetricType> metricTypes = Collections.singletonList(MetricType.all); // include all metrics by default
+    try {
+      if (types.size() > 0) {
+        metricTypes = types.stream().map(String::trim).map(MetricType::valueOf).collect(Collectors.toList());
+      }
+    } catch (IllegalArgumentException e) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Invalid metric type in: " + types + " specified. Must be one of (all, meter, timer, histogram, counter, gauge)", e);
+    }
+    return metricTypes;
+  }
+
+  @Override
+  public String getDescription() {
+    return "A handler to return all the metrics gathered by Solr";
+  }
+
+  enum MetricType {
+    histogram(Histogram.class),
+    meter(Meter.class),
+    timer(Timer.class),
+    counter(Counter.class),
+    gauge(Gauge.class),
+    all(null);
+
+    private final Class klass;
+
+    MetricType(Class klass) {
+      this.klass = klass;
+    }
+
+    public MetricFilter asMetricFilter() {
+      return (name, metric) -> klass == null || klass.isInstance(metric);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aa9b02bb/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java b/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java
index bf3ee1b..18307ee 100644
--- a/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java
@@ -19,18 +19,12 @@ package org.apache.solr.metrics;
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.Collection;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.Map;
 import java.lang.invoke.MethodHandles;
-import java.util.concurrent.ConcurrentHashMap;
 
-import com.codahale.metrics.MetricRegistry;
 import org.apache.solr.core.NodeConfig;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrInfoMBean;
-import org.apache.solr.core.SolrResourceLoader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aa9b02bb/solr/core/src/java/org/apache/solr/metrics/SolrMetricReporter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrMetricReporter.java b/solr/core/src/java/org/apache/solr/metrics/SolrMetricReporter.java
index 60d3b58..cfe5d67 100644
--- a/solr/core/src/java/org/apache/solr/metrics/SolrMetricReporter.java
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrMetricReporter.java
@@ -17,7 +17,6 @@
 package org.apache.solr.metrics;
 
 import java.io.Closeable;
-import java.util.Locale;
 
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.util.SolrPluginUtils;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aa9b02bb/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java b/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java
index d0e5c9a..425be38 100644
--- a/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java
+++ b/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java
@@ -46,6 +46,7 @@ public interface PermissionNameProvider {
     SCHEMA_EDIT_PERM("schema-edit", "*"),
     SECURITY_EDIT_PERM("security-edit", null),
     SECURITY_READ_PERM("security-read", null),
+    METRICS_READ_PERM("metrics-read", null),
     ALL("all", unmodifiableSet(new HashSet<>(asList("*", null))))
     ;
     final String name;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aa9b02bb/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
index 1c7b839..cfd90cb 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
@@ -47,8 +47,6 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import com.codahale.metrics.MetricRegistry;
-import com.codahale.metrics.SharedMetricRegistries;
 import com.codahale.metrics.jvm.BufferPoolMetricSet;
 import com.codahale.metrics.jvm.ClassLoadingGaugeSet;
 import com.codahale.metrics.jvm.FileDescriptorRatioGauge;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aa9b02bb/solr/core/src/java/org/apache/solr/servlet/SolrMetricsServletContextListener.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/SolrMetricsServletContextListener.java b/solr/core/src/java/org/apache/solr/servlet/SolrMetricsServletContextListener.java
deleted file mode 100644
index 041ef11..0000000
--- a/solr/core/src/java/org/apache/solr/servlet/SolrMetricsServletContextListener.java
+++ /dev/null
@@ -1,29 +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.solr.servlet;
-
-import com.codahale.metrics.MetricRegistry;
-import com.codahale.metrics.SharedMetricRegistries;
-import com.codahale.metrics.servlets.MetricsServlet;
-
-public class SolrMetricsServletContextListener extends MetricsServlet.ContextListener {
-  @Override
-  protected MetricRegistry getMetricRegistry() {
-    return SharedMetricRegistries.getOrCreate("solr.jetty");
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aa9b02bb/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java b/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java
new file mode 100644
index 0000000..62f5776
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java
@@ -0,0 +1,144 @@
+/*
+ * 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.List;
+import java.util.Map;
+import java.util.SortedSet;
+import java.util.concurrent.TimeUnit;
+
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.Meter;
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricFilter;
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.Snapshot;
+import com.codahale.metrics.Timer;
+import org.apache.solr.common.util.NamedList;
+
+/**
+ * Metrics specific utility functions.
+ */
+public class MetricUtils {
+
+  /**
+   * Adds metrics from a Timer to a NamedList, using well-known names.
+   * @param lst The NamedList to add the metrics data to
+   * @param timer The Timer to extract the metrics from
+   */
+  public static void addMetrics(NamedList<Object> lst, Timer timer) {
+    Snapshot snapshot = timer.getSnapshot();
+    lst.add("avgRequestsPerSecond", timer.getMeanRate());
+    lst.add("5minRateRequestsPerSecond", timer.getFiveMinuteRate());
+    lst.add("15minRateRequestsPerSecond", timer.getFifteenMinuteRate());
+    lst.add("avgTimePerRequest", nsToMs(snapshot.getMean()));
+    lst.add("medianRequestTime", nsToMs(snapshot.getMedian()));
+    lst.add("75thPcRequestTime", nsToMs(snapshot.get75thPercentile()));
+    lst.add("95thPcRequestTime", nsToMs(snapshot.get95thPercentile()));
+    lst.add("99thPcRequestTime", nsToMs(snapshot.get99thPercentile()));
+    lst.add("999thPcRequestTime", nsToMs(snapshot.get999thPercentile()));
+  }
+
+  /**
+   * Converts a double representing nanoseconds to a double representing milliseconds.
+   *
+   * @param ns the amount of time in nanoseconds
+   * @return the amount of time in milliseconds
+   */
+  static double nsToMs(double ns) {
+    return ns / TimeUnit.MILLISECONDS.toNanos(1);
+  }
+
+  /**
+   * Returns a NamedList respresentation of the given metric registry. Only those metrics
+   * are converted to NamedList which match at least one of the given MetricFilter instances.
+   *
+   * @param registry      the {@link MetricRegistry} to be converted to NamedList
+   * @param metricFilters a list of {@link MetricFilter} instances
+   * @return a {@link NamedList}
+   */
+  public static NamedList toNamedList(MetricRegistry registry, List<MetricFilter> metricFilters) {
+    NamedList response = new NamedList();
+    Map<String, Metric> metrics = registry.getMetrics();
+    SortedSet<String> names = registry.getNames();
+    names.stream().filter(s -> metricFilters.stream().anyMatch(metricFilter -> metricFilter.matches(s, metrics.get(s)))).forEach(n -> {
+      Metric metric = metrics.get(n);
+      if (metric instanceof Counter) {
+        Counter counter = (Counter) metric;
+        response.add(n, counterToNamedList(counter));
+      } else if (metric instanceof Gauge) {
+        Gauge gauge = (Gauge) metric;
+        response.add(n, gaugeToNamedList(gauge));
+      } else if (metric instanceof Meter) {
+        Meter meter = (Meter) metric;
+        response.add(n, meterToNamedList(meter));
+      } else if (metric instanceof Timer) {
+        Timer timer = (Timer) metric;
+        response.add(n, timerToNamedList(timer));
+      } else if (metric instanceof Histogram) {
+        Histogram histogram = (Histogram) metric;
+        response.add(n, histogramToNamedList(histogram));
+      }
+    });
+    return response;
+  }
+
+  static NamedList histogramToNamedList(Histogram histogram) {
+    NamedList response = new NamedList();
+    Snapshot snapshot = histogram.getSnapshot();
+    response.add("requests", histogram.getCount());
+    response.add("minTime", nsToMs(snapshot.getMin()));
+    response.add("maxTime", nsToMs(snapshot.getMax()));
+    response.add("avgTimePerRequest", nsToMs(snapshot.getMean()));
+    response.add("medianRequestTime", nsToMs(snapshot.getMedian()));
+    response.add("75thPcRequestTime", nsToMs(snapshot.get75thPercentile()));
+    response.add("95thPcRequestTime", nsToMs(snapshot.get95thPercentile()));
+    response.add("99thPcRequestTime", nsToMs(snapshot.get99thPercentile()));
+    response.add("999thPcRequestTime", nsToMs(snapshot.get999thPercentile()));
+    return response;
+  }
+
+  static NamedList timerToNamedList(Timer timer) {
+    NamedList response = new NamedList();
+    addMetrics(response, timer);
+    return response;
+  }
+
+  static NamedList meterToNamedList(Meter meter) {
+    NamedList response = new NamedList();
+    response.add("requests", meter.getCount());
+    response.add("avgRequestsPerSecond", meter.getMeanRate());
+    response.add("1minRateRequestsPerSecond", meter.getOneMinuteRate());
+    response.add("5minRateRequestsPerSecond", meter.getFiveMinuteRate());
+    response.add("15minRateRequestsPerSecond", meter.getFifteenMinuteRate());
+    return response;
+  }
+
+  static NamedList gaugeToNamedList(Gauge gauge) {
+    NamedList response = new NamedList();
+    response.add("value", gauge.getValue());
+    return response;
+  }
+
+  static NamedList counterToNamedList(Counter counter) {
+    NamedList response = new NamedList();
+    response.add("requests", counter.getCount());
+    return response;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aa9b02bb/solr/core/src/java/org/apache/solr/util/stats/TimerUtils.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/stats/TimerUtils.java b/solr/core/src/java/org/apache/solr/util/stats/TimerUtils.java
deleted file mode 100644
index 243c1ee..0000000
--- a/solr/core/src/java/org/apache/solr/util/stats/TimerUtils.java
+++ /dev/null
@@ -1,58 +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.solr.util.stats;
-
-import java.util.concurrent.TimeUnit;
-
-import com.codahale.metrics.Snapshot;
-import com.codahale.metrics.Timer;
-import org.apache.solr.common.util.NamedList;
-
-/**
- * Solr specific {@link Timer} utility functions.
- */
-public class TimerUtils {
-
-  /**
-   * Adds metrics from a Timer to a NamedList, using well-known names.
-   * @param lst The NamedList to add the metrics data to
-   * @param timer The Timer to extract the metrics from
-   */
-  public static void addMetrics(NamedList<Object> lst, Timer timer) {
-    Snapshot snapshot = timer.getSnapshot();
-    lst.add("avgRequestsPerSecond", timer.getMeanRate());
-    lst.add("5minRateRequestsPerSecond", timer.getFiveMinuteRate());
-    lst.add("15minRateRequestsPerSecond", timer.getFifteenMinuteRate());
-    lst.add("avgTimePerRequest", nsToMs(snapshot.getMean()));
-    lst.add("medianRequestTime", nsToMs(snapshot.getMedian()));
-    lst.add("75thPcRequestTime", nsToMs(snapshot.get75thPercentile()));
-    lst.add("95thPcRequestTime", nsToMs(snapshot.get95thPercentile()));
-    lst.add("99thPcRequestTime", nsToMs(snapshot.get99thPercentile()));
-    lst.add("999thPcRequestTime", nsToMs(snapshot.get999thPercentile()));
-  }
-
-  /**
-   * Converts a double representing nanoseconds to a double representing milliseconds.
-   *
-   * @param ns the amount of time in nanoseconds
-   * @return the amount of time in milliseconds
-   */
-  static double nsToMs(double ns) {
-    return ns / TimeUnit.MILLISECONDS.toNanos(1);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aa9b02bb/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
new file mode 100644
index 0000000..83ce74f
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
@@ -0,0 +1,95 @@
+/*
+ * 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.handler.admin;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.response.SolrQueryResponse;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test for {@link MetricsHandler}
+ */
+public class MetricsHandlerTest extends SolrTestCaseJ4 {
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    initCore("solrconfig.xml", "schema.xml");
+  }
+
+  @Test
+  public void test() throws Exception {
+    MetricsHandler handler = new MetricsHandler(h.getCoreContainer());
+
+    SolrQueryResponse resp = new SolrQueryResponse();
+    handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json"), resp);
+    NamedList values = resp.getValues();
+    assertNotNull(values.get("metrics"));
+    values = (NamedList) values.get("metrics");
+    System.out.println(values);
+    assertNotNull(values.get("solr.jetty"));
+    assertNotNull(values.get("solr.jvm"));
+    assertNotNull(values.get("solr.http"));
+    assertNotNull(values.get("solr.node"));
+    NamedList nl = (NamedList) values.get("solr.core.collection1");
+    assertNotNull(nl);
+    assertNotNull(nl.get("newSearcherErrors")); // counter type
+    assertNotNull(((NamedList) nl.get("newSearcherErrors")).get("requests"));
+    assertEquals(0L, ((NamedList) nl.get("newSearcherErrors")).get("requests"));
+    nl = (NamedList) values.get("solr.node");
+    assertNotNull(nl.get("QUERYHANDLER./admin/authorization.clientErrors")); // timer type
+    assertEquals(5, ((NamedList) nl.get("QUERYHANDLER./admin/authorization.clientErrors")).size());
+
+    resp = new SolrQueryResponse();
+    handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", "group", "jvm,jetty"), resp);
+    values = resp.getValues();
+    assertNotNull(values.get("metrics"));
+    values = (NamedList) values.get("metrics");
+    assertEquals(2, values.size());
+    assertNotNull(values.get("solr.jetty"));
+    assertNotNull(values.get("solr.jvm"));
+
+    resp = new SolrQueryResponse();
+    handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", "group", "jvm,jetty"), resp);
+    values = resp.getValues();
+    assertNotNull(values.get("metrics"));
+    values = (NamedList) values.get("metrics");
+    assertEquals(2, values.size());
+    assertNotNull(values.get("solr.jetty"));
+    assertNotNull(values.get("solr.jvm"));
+
+    resp = new SolrQueryResponse();
+    handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", "group", "jvm", "group", "jetty"), resp);
+    values = resp.getValues();
+    assertNotNull(values.get("metrics"));
+    values = (NamedList) values.get("metrics");
+    assertEquals(2, values.size());
+    assertNotNull(values.get("solr.jetty"));
+    assertNotNull(values.get("solr.jvm"));
+
+    resp = new SolrQueryResponse();
+    handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", "group", "node", "type", "counter"), resp);
+    values = resp.getValues();
+    assertNotNull(values.get("metrics"));
+    values = (NamedList) values.get("metrics");
+    assertEquals(1, values.size());
+    assertNotNull(values.get("solr.node"));
+    assertNull(values.get("QUERYHANDLER./admin/authorization.errors")); // this is a timer node
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aa9b02bb/solr/core/src/test/org/apache/solr/util/stats/MetricUtilsTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/util/stats/MetricUtilsTest.java b/solr/core/src/test/org/apache/solr/util/stats/MetricUtilsTest.java
new file mode 100644
index 0000000..31e8154
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/util/stats/MetricUtilsTest.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.util.stats;
+
+import java.util.concurrent.TimeUnit;
+
+import com.codahale.metrics.Snapshot;
+import com.codahale.metrics.Timer;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.junit.Test;
+
+public class MetricUtilsTest extends SolrTestCaseJ4 {
+
+  @Test
+  public void testSolrTimerGetSnapshot() {
+    // create a timer with up to 100 data points
+    final Timer timer = new Timer();
+    final int iterations = random().nextInt(100);
+    for (int i = 0; i < iterations; ++i) {
+      timer.update(random().nextInt(), TimeUnit.NANOSECONDS);
+    }
+    // obtain timer metrics
+    final NamedList<Object> lst = new SimpleOrderedMap<>();
+    MetricUtils.addMetrics(lst, timer);
+    // check that expected metrics were obtained
+    assertEquals(lst.size(), 9);
+    final Snapshot snapshot = timer.getSnapshot();
+    // cannot test avgRequestsPerMinute directly because mean rate changes as time increases!
+    // assertEquals(lst.get("avgRequestsPerSecond"), timer.getMeanRate());
+    assertEquals(lst.get("5minRateRequestsPerSecond"), timer.getFiveMinuteRate());
+    assertEquals(lst.get("15minRateRequestsPerSecond"), timer.getFifteenMinuteRate());
+    assertEquals(lst.get("avgTimePerRequest"), MetricUtils.nsToMs(snapshot.getMean()));
+    assertEquals(lst.get("medianRequestTime"), MetricUtils.nsToMs(snapshot.getMedian()));
+    assertEquals(lst.get("75thPcRequestTime"), MetricUtils.nsToMs(snapshot.get75thPercentile()));
+    assertEquals(lst.get("95thPcRequestTime"), MetricUtils.nsToMs(snapshot.get95thPercentile()));
+    assertEquals(lst.get("99thPcRequestTime"), MetricUtils.nsToMs(snapshot.get99thPercentile()));
+    assertEquals(lst.get("999thPcRequestTime"), MetricUtils.nsToMs(snapshot.get999thPercentile()));
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aa9b02bb/solr/core/src/test/org/apache/solr/util/stats/TimerUtilsTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/util/stats/TimerUtilsTest.java b/solr/core/src/test/org/apache/solr/util/stats/TimerUtilsTest.java
deleted file mode 100644
index 851f768..0000000
--- a/solr/core/src/test/org/apache/solr/util/stats/TimerUtilsTest.java
+++ /dev/null
@@ -1,58 +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.solr.util.stats;
-
-import java.util.concurrent.TimeUnit;
-
-import com.codahale.metrics.Snapshot;
-import com.codahale.metrics.Timer;
-import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.common.util.NamedList;
-import org.apache.solr.common.util.SimpleOrderedMap;
-import org.junit.Test;
-
-public class TimerUtilsTest extends SolrTestCaseJ4 {
-
-  @Test
-  public void testSolrTimerGetSnapshot() {
-    // create a timer with up to 100 data points
-    final Timer timer = new Timer();
-    final int iterations = random().nextInt(100);
-    for (int i = 0; i < iterations; ++i) {
-      timer.update(random().nextInt(), TimeUnit.NANOSECONDS);
-    }
-    // obtain timer metrics
-    final NamedList<Object> lst = new SimpleOrderedMap<>();
-    TimerUtils.addMetrics(lst, timer);
-    // check that expected metrics were obtained
-    assertEquals(lst.size(), 9);
-    final Snapshot snapshot = timer.getSnapshot();
-    // cannot test avgRequestsPerMinute directly because mean rate changes as time increases!
-    // assertEquals(lst.get("avgRequestsPerSecond"), timer.getMeanRate());
-    assertEquals(lst.get("5minRateRequestsPerSecond"), timer.getFiveMinuteRate());
-    assertEquals(lst.get("15minRateRequestsPerSecond"), timer.getFifteenMinuteRate());
-    assertEquals(lst.get("avgTimePerRequest"), TimerUtils.nsToMs(snapshot.getMean()));
-    assertEquals(lst.get("medianRequestTime"), TimerUtils.nsToMs(snapshot.getMedian()));
-    assertEquals(lst.get("75thPcRequestTime"), TimerUtils.nsToMs(snapshot.get75thPercentile()));
-    assertEquals(lst.get("95thPcRequestTime"), TimerUtils.nsToMs(snapshot.get95thPercentile()));
-    assertEquals(lst.get("99thPcRequestTime"), TimerUtils.nsToMs(snapshot.get99thPercentile()));
-    assertEquals(lst.get("999thPcRequestTime"), TimerUtils.nsToMs(snapshot.get999thPercentile()));
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aa9b02bb/solr/licenses/metrics-json-3.1.2.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/metrics-json-3.1.2.jar.sha1 b/solr/licenses/metrics-json-3.1.2.jar.sha1
deleted file mode 100644
index 93168e9..0000000
--- a/solr/licenses/metrics-json-3.1.2.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-88d9e57e1ef6431109d4030c717cf5f927900fd9

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aa9b02bb/solr/licenses/metrics-servlets-3.1.2.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/metrics-servlets-3.1.2.jar.sha1 b/solr/licenses/metrics-servlets-3.1.2.jar.sha1
deleted file mode 100644
index ba941e5..0000000
--- a/solr/licenses/metrics-servlets-3.1.2.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-f552e662d790b805f01a8b2997dc4ae9409cd4c8

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aa9b02bb/solr/server/ivy.xml
----------------------------------------------------------------------
diff --git a/solr/server/ivy.xml b/solr/server/ivy.xml
index 2f3c39c..8dc645a 100644
--- a/solr/server/ivy.xml
+++ b/solr/server/ivy.xml
@@ -34,13 +34,8 @@
     <dependency org="org.slf4j" name="slf4j-log4j12" rev="${/org.slf4j/slf4j-log4j12}" conf="logging"/>
 
     <dependency org="io.dropwizard.metrics" name="metrics-core" rev="${/io.dropwizard.metrics/metrics-core}" conf="metrics" />
-    <dependency org="io.dropwizard.metrics" name="metrics-servlets" rev="${/io.dropwizard.metrics/metrics-servlets}" conf="metrics" />
-    <dependency org="io.dropwizard.metrics" name="metrics-json" rev="${/io.dropwizard.metrics/metrics-json}" conf="metrics" />
     <dependency org="io.dropwizard.metrics" name="metrics-jetty9" rev="${/io.dropwizard.metrics/metrics-jetty9}" conf="metrics" />
     <dependency org="io.dropwizard.metrics" name="metrics-jvm" rev="${/io.dropwizard.metrics/metrics-jvm}" conf="metrics" />
-    <dependency org="com.fasterxml.jackson.core" name="jackson-core" rev="${/com.fasterxml.jackson.core/jackson-core}" conf="metrics"/>
-    <dependency org="com.fasterxml.jackson.core" name="jackson-databind" rev="${/com.fasterxml.jackson.core/jackson-databind}" conf="metrics"/>
-    <dependency org="com.fasterxml.jackson.core" name="jackson-annotations" rev="${/com.fasterxml.jackson.core/jackson-annotations}" conf="metrics"/>
 
     <dependency org="org.eclipse.jetty" name="jetty-continuation" rev="${/org.eclipse.jetty/jetty-continuation}" conf="jetty"/>
     <dependency org="org.eclipse.jetty" name="jetty-deploy" rev="${/org.eclipse.jetty/jetty-deploy}" conf="jetty"/>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aa9b02bb/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java b/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java
index 411d40d..7cf27d2 100644
--- a/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java
+++ b/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java
@@ -180,13 +180,15 @@ public interface CommonParams {
   String AUTHZ_PATH = "/admin/authorization";
   String AUTHC_PATH = "/admin/authentication";
   String ZK_PATH = "/admin/zookeeper";
+  String METRICS_PATH = "/admin/metrics";
 
   Set<String> ADMIN_PATHS = new HashSet<>(Arrays.asList(
       CORES_HANDLER_PATH,
       COLLECTIONS_HANDLER_PATH,
       CONFIGSETS_HANDLER_PATH,
       AUTHC_PATH,
-      AUTHZ_PATH));
+      AUTHZ_PATH,
+      METRICS_PATH));
 
   /** valid values for: <code>echoParams</code> */
   enum EchoParamStyle {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aa9b02bb/solr/webapp/web/WEB-INF/web.xml
----------------------------------------------------------------------
diff --git a/solr/webapp/web/WEB-INF/web.xml b/solr/webapp/web/WEB-INF/web.xml
index 935e411..5278ae5 100644
--- a/solr/webapp/web/WEB-INF/web.xml
+++ b/solr/webapp/web/WEB-INF/web.xml
@@ -75,23 +75,6 @@
     <url-pattern>/*</url-pattern>
   </filter-mapping>
 
-  <listener>
-    <listener-class>org.apache.solr.servlet.SolrMetricsServletContextListener</listener-class>
-  </listener>
-
-  <servlet>
-    <servlet-name>metrics</servlet-name>
-    <servlet-class>com.codahale.metrics.servlets.MetricsServlet</servlet-class>
-    <init-param>
-      <param-name>show-jvm-metrics</param-name>
-      <param-value>true</param-value>
-    </init-param>
-  </servlet>
-  <servlet-mapping>
-    <servlet-name>metrics</servlet-name>
-    <url-pattern>/admin/metrics/*</url-pattern>
-  </servlet-mapping>
-
   <servlet>
     <servlet-name>LoadAdminUI</servlet-name>
     <servlet-class>org.apache.solr.servlet.LoadAdminUiServlet</servlet-class>