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 2017/01/19 15:30:29 UTC

lucene-solr:jira/solr-9857: SOLR-9857 Change SolrReporter / MetricsCollectorHandler to use regular UpdateRequest-s so that SolrReporter can be reused in other contexts. Change also MetricUtils so that it's suitable for producing either NamedList or SolrI

Repository: lucene-solr
Updated Branches:
  refs/heads/jira/solr-9857 ae16cfd93 -> 197bf0013


SOLR-9857 Change SolrReporter / MetricsCollectorHandler to use regular UpdateRequest-s
so that SolrReporter can be reused in other contexts. Change also MetricUtils so that
it's suitable for producing either NamedList or SolrInputDocuments.


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

Branch: refs/heads/jira/solr-9857
Commit: 197bf00134bb7059e3b1119874182d2cf52d6346
Parents: ae16cfd
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Thu Jan 19 16:29:02 2017 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Thu Jan 19 16:29:02 2017 +0100

----------------------------------------------------------------------
 .../org/apache/solr/core/CoreContainer.java     |   2 +
 .../handler/admin/MetricsCollectorHandler.java  | 138 ++++++++++--
 .../solr/handler/admin/MetricsHandler.java      |   2 +-
 .../reporters/solr/SolrReplicaReporter.java     |   3 +-
 .../metrics/reporters/solr/SolrReporter.java    |  74 ++++++-
 .../org/apache/solr/util/stats/MetricUtils.java | 218 ++++++++++++-------
 .../reporters/solr/SolrReplicaReporterTest.java |   2 +-
 .../apache/solr/util/stats/MetricUtilsTest.java |   2 +-
 8 files changed, 332 insertions(+), 109 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/197bf001/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 c622552..c80816b 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -502,6 +502,8 @@ public class CoreContainer {
     configSetsHandler = createHandler(CONFIGSETS_HANDLER_PATH, cfg.getConfigSetsHandlerClass(), ConfigSetsHandler.class);
     metricsHandler = createHandler(METRICS_PATH, MetricsHandler.class.getName(), MetricsHandler.class);
     metricsCollectorHandler = createHandler(MetricsCollectorHandler.HANDLER_PATH, MetricsCollectorHandler.class.getName(), MetricsCollectorHandler.class);
+    // nocommit add some configuration here
+    metricsCollectorHandler.init(null);
     containerHandlers.put(AUTHZ_PATH, securityConfHandler);
     securityConfHandler.initializeMetrics(metricManager, SolrInfoMBean.Group.node.toString(), AUTHZ_PATH);
     containerHandlers.put(AUTHC_PATH, securityConfHandler);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/197bf001/solr/core/src/java/org/apache/solr/handler/admin/MetricsCollectorHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/MetricsCollectorHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/MetricsCollectorHandler.java
index 9c963cc..c504273 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/MetricsCollectorHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/MetricsCollectorHandler.java
@@ -16,24 +16,44 @@
  */
 package org.apache.solr.handler.admin;
 
+import java.io.IOException;
 import java.lang.invoke.MethodHandles;
+import java.util.HashMap;
 import java.util.Map;
 
 import com.codahale.metrics.MetricRegistry;
+import org.apache.solr.client.solrj.io.stream.metrics.Metric;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.ContentStream;
 import org.apache.solr.common.util.JavaBinCodec;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.CoreContainer;
+import org.apache.solr.handler.loader.ContentStreamLoader;
 import org.apache.solr.handler.RequestHandlerBase;
+import org.apache.solr.handler.loader.CSVLoader;
+import org.apache.solr.handler.loader.JavabinLoader;
+import org.apache.solr.handler.loader.JsonLoader;
+import org.apache.solr.handler.loader.XMLLoader;
 import org.apache.solr.metrics.AggregateMetric;
 import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.metrics.reporters.solr.SolrReporter;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.update.AddUpdateCommand;
+import org.apache.solr.update.CommitUpdateCommand;
+import org.apache.solr.update.DeleteUpdateCommand;
+import org.apache.solr.update.MergeIndexesCommand;
+import org.apache.solr.update.RollbackUpdateCommand;
+import org.apache.solr.update.processor.UpdateRequestProcessor;
+import org.apache.solr.util.stats.MetricUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- *
+ * Handler to collect and aggregate metric reports.
  */
 public class MetricsCollectorHandler extends RequestHandlerBase {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@@ -42,43 +62,117 @@ public class MetricsCollectorHandler extends RequestHandlerBase {
 
   private final CoreContainer coreContainer;
   private final SolrMetricManager metricManager;
+  private final Map<String, ContentStreamLoader> registry = new HashMap<>();
+  private SolrParams params;
 
   public MetricsCollectorHandler(final CoreContainer coreContainer) {
     this.coreContainer = coreContainer;
     this.metricManager = coreContainer.getMetricManager();
+
+  }
+
+  @Override
+  public void init(NamedList initArgs) {
+    super.init(initArgs);
+    if (initArgs != null) {
+      params = SolrParams.toSolrParams(initArgs);
+    } else {
+      params = new ModifiableSolrParams();
+    }
+    registry.put("application/xml", new XMLLoader().init(params) );
+    registry.put("application/json", new JsonLoader().init(params) );
+    registry.put("application/csv", new CSVLoader().init(params) );
+    registry.put("application/javabin", new JavabinLoader().init(params) );
+    registry.put("text/csv", registry.get("application/csv") );
+    registry.put("text/xml", registry.get("application/xml") );
+    registry.put("text/json", registry.get("application/json"));
   }
 
   @Override
   public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
     log.info("#### " + req.toString());
     for (ContentStream cs : req.getContentStreams()) {
-      // only javabin supported
-      if (!cs.getContentType().equals("application/javabin")) {
-        log.warn("Invalid content type, skipping " + cs.getSourceInfo() + ": " + cs.getContentType());
+      if (cs.getContentType() == null) {
+        log.warn("Missing content type - ignoring");
+        continue;
+      }
+      ContentStreamLoader loader = registry.get(cs.getContentType());
+      if (loader == null) {
+        throw new SolrException(SolrException.ErrorCode.UNSUPPORTED_MEDIA_TYPE, "Unsupported content type for stream: " + cs.getSourceInfo() + ", contentType=" + cs.getContentType());
+      }
+      String id = req.getParams().get(SolrReporter.REPORTER_ID);
+      String group = req.getParams().get(SolrReporter.GROUP_ID);
+      if (id == null || group == null) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Missing " + SolrReporter.REPORTER_ID +
+            " or " + SolrReporter.GROUP_ID + " in request params: " + req.getParamString());
       }
-      JavaBinCodec codec = new JavaBinCodec();
-      NamedList nl = (NamedList)codec.unmarshal(cs.getStream());
-      String id = (String)nl.get("id");
-      String group = (String)nl.get("group");
       MetricRegistry registry = metricManager.registry(group);
-      // values are tuples of String / NamedList<Number>
-      NamedList values = (NamedList)nl.get("values");
-      values.forEach((k, v) -> {
-        ((NamedList)v).forEach((k1, v1) -> {
-          String key = MetricRegistry.name(k.toString(), k1.toString());
-          AggregateMetric metric = (AggregateMetric)registry.getMetrics().get(key);
-          if (metric == null) {
-            metric = new AggregateMetric();
-            registry.register(key, metric);
-          }
-          metric.set(id, ((Number)v1).doubleValue());
-        });
-      });
+      loader.load(req, rsp, cs, new MetricUpdateProcessor(registry, id, group));
     }
   }
 
   @Override
   public String getDescription() {
-    return null;
+    return "Handler for collecting and aggregating metric reports.";
+  }
+
+  private static class MetricUpdateProcessor extends UpdateRequestProcessor {
+    private final MetricRegistry registry;
+    private final String id;
+    private final String group;
+
+    public MetricUpdateProcessor(MetricRegistry registry, String id, String group) {
+      super(null);
+      this.registry = registry;
+      this.id = id;
+      this.group = group;
+    }
+
+    @Override
+    public void processAdd(AddUpdateCommand cmd) throws IOException {
+      SolrInputDocument doc = cmd.solrDoc;
+      if (doc == null) {
+        return;
+      }
+      String metricName = (String)doc.getFieldValue("name");
+      if (metricName == null) {
+        log.warn("Missing metric 'name' field in document, skipping: " + doc);
+        return;
+      }
+      doc.remove(MetricUtils.NAME);
+      // already known
+      doc.remove(SolrReporter.REPORTER_ID);
+      doc.remove(SolrReporter.GROUP_ID);
+      // remaining fields should only contain numeric values
+      doc.forEach(f -> {
+        String key = MetricRegistry.name(metricName, f.getName());
+        AggregateMetric metric = (AggregateMetric)registry.getMetrics().get(key);
+        if (metric == null) {
+          metric = new AggregateMetric();
+          registry.register(key, metric);
+        }
+        metric.set(id, ((Number)f.getFirstValue()).doubleValue());
+      });
+    }
+
+    @Override
+    public void processDelete(DeleteUpdateCommand cmd) throws IOException {
+      super.processDelete(cmd);
+    }
+
+    @Override
+    public void processMergeIndexes(MergeIndexesCommand cmd) throws IOException {
+      throw new UnsupportedOperationException("processMergeIndexes");
+    }
+
+    @Override
+    public void processCommit(CommitUpdateCommand cmd) throws IOException {
+      throw new UnsupportedOperationException("processCommit");
+    }
+
+    @Override
+    public void processRollback(RollbackUpdateCommand cmd) throws IOException {
+      throw new UnsupportedOperationException("processRollback");
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/197bf001/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
index 40a309a..56a37fe 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java
@@ -79,7 +79,7 @@ public class MetricsHandler extends RequestHandlerBase implements PermissionName
     NamedList response = new NamedList();
     for (String registryName : requestedRegistries) {
       MetricRegistry registry = metricManager.registry(registryName);
-      response.add(registryName, MetricUtils.toNamedList(registry, metricFilters, mustMatchFilter, false));
+      response.add(registryName, MetricUtils.toNamedList(registry, metricFilters, mustMatchFilter, false, null));
     }
     rsp.getValues().add("metrics", response);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/197bf001/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrReplicaReporter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrReplicaReporter.java b/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrReplicaReporter.java
index 0e335dc..bddb1f0 100644
--- a/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrReplicaReporter.java
+++ b/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrReplicaReporter.java
@@ -30,7 +30,7 @@ import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.metrics.SolrMetricReporter;
 
 /**
- *
+ * This class reports selected metrics from replicas to a shard leader.
  */
 public class SolrReplicaReporter extends SolrMetricReporter {
   public static final String LEADER_REGISTRY = "leaderRegistry";
@@ -96,6 +96,7 @@ public class SolrReplicaReporter extends SolrMetricReporter {
         .withHandler(handler)
         .filter(filter)
         .withId(id)
+        .cloudClient(false) // we want to send reports specifically to a selected leader instance
         .withGroup(leaderRegistry)
         .build(core.getCoreDescriptor().getCoreContainer().getUpdateShardHandler().getHttpClient(), new LeaderUrlSupplier(core));
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/197bf001/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrReporter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrReporter.java b/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrReporter.java
index 50fedf4..ddbc2c2 100644
--- a/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrReporter.java
+++ b/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrReporter.java
@@ -19,7 +19,9 @@ package org.apache.solr.metrics.reporters.solr;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.SortedMap;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Supplier;
@@ -36,6 +38,10 @@ import org.apache.http.client.HttpClient;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.io.SolrClientCache;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.JavaBinCodec;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.handler.admin.MetricsCollectorHandler;
@@ -58,6 +64,8 @@ public class SolrReporter extends ScheduledReporter {
     private MetricFilter filter;
     private String handler;
     private boolean skipHistograms;
+    private boolean cloudClient;
+    private SolrParams params;
 
     public static Builder forRegistry(MetricRegistry registry) {
       return new Builder(registry);
@@ -69,6 +77,28 @@ public class SolrReporter extends ScheduledReporter {
       this.durationUnit = TimeUnit.MILLISECONDS;
       this.filter = MetricFilter.ALL;
       this.skipHistograms = false;
+      this.cloudClient = false;
+      this.params = null;
+    }
+
+    /**
+     * Additional {@link SolrParams} to add to every request.
+     * @param params additional params
+     * @return {@code this}
+     */
+    public Builder withSolrParams(SolrParams params) {
+      this.params = params;
+      return this;
+    }
+    /**
+     * If true then use {@link org.apache.solr.client.solrj.impl.CloudSolrClient} for communication.
+     * Default is false.
+     * @param cloudClient use CloudSolrClient when true, {@link org.apache.solr.client.solrj.impl.HttpSolrClient} otherwise.
+     * @return {@code this}
+     */
+    public Builder cloudClient(boolean cloudClient) {
+      this.cloudClient = cloudClient;
+      return this;
     }
 
     /**
@@ -157,11 +187,15 @@ public class SolrReporter extends ScheduledReporter {
      * @return configured instance of reporter
      */
     public SolrReporter build(HttpClient client, Supplier<String> urlProvider) {
-      return new SolrReporter(client, urlProvider, registry, handler, id, group, rateUnit, durationUnit, filter, skipHistograms);
+      return new SolrReporter(client, urlProvider, registry, handler, id, group, rateUnit, durationUnit,
+          filter, params, skipHistograms, cloudClient);
     }
 
   }
 
+  public static final String REPORTER_ID = "solrReporterId";
+  public static final String GROUP_ID = "solrGroupId";
+
   private String id;
   private String group;
   private String handler;
@@ -170,10 +204,13 @@ public class SolrReporter extends ScheduledReporter {
   private List<MetricFilter> filters;
   private MetricRegistry visibleRegistry;
   private boolean skipHistograms;
+  private boolean cloudClient;
+  private ModifiableSolrParams params;
+  private Map<String, Object> metadata;
 
   public SolrReporter(HttpClient httpClient, Supplier<String> urlProvider, MetricRegistry registry, String handler,
                       String id, String group, TimeUnit rateUnit, TimeUnit durationUnit, MetricFilter filter,
-                      boolean skipHistograms) {
+                      SolrParams params, boolean skipHistograms, boolean cloudClient) {
     super(registry, "solr-reporter", filter, rateUnit, durationUnit);
     this.urlProvider = urlProvider;
     this.id = id;
@@ -190,6 +227,19 @@ public class SolrReporter extends ScheduledReporter {
     }
     this.filters = Collections.singletonList(filter);
     this.skipHistograms = skipHistograms;
+    this.cloudClient = cloudClient;
+    this.params = new ModifiableSolrParams();
+    this.params.set(REPORTER_ID, id);
+    this.params.set(GROUP_ID, group);
+    // allow overrides to take precedence
+    if (params != null) {
+      this.params.add(params);
+    }
+    metadata = new HashMap<>();
+    metadata.put(REPORTER_ID, id);
+    if (group != null) {
+      metadata.put(GROUP_ID, group);
+    }
   }
 
   @Override
@@ -199,14 +249,18 @@ public class SolrReporter extends ScheduledReporter {
     if (url == null) {
       return;
     }
-    NamedList nl = MetricUtils.toNamedList(visibleRegistry, filters, MetricFilter.ALL, skipHistograms);
-    NamedList report = new NamedList();
-    report.add("id", id);
-    report.add("group", group);
-    report.add("values", nl);
-    JavaBinCodec codec = new JavaBinCodec();
-    SolrClient solr = clientCache.getHttpSolrClient(url);
-    MetricsReportRequest req = new MetricsReportRequest(handler, null, report);
+
+    SolrClient solr;
+    if (cloudClient) {
+      solr = clientCache.getCloudSolrClient(url);
+    } else {
+      solr = clientCache.getHttpSolrClient(url);
+    }
+    UpdateRequest req = new UpdateRequest(handler);
+    req.setParams(params);
+    MetricUtils.toSolrInputDocuments(visibleRegistry, filters, MetricFilter.ALL,
+        skipHistograms, metadata, doc -> req.add(doc));
+
     try {
       solr.request(req);
     } catch (SolrServerException sse) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/197bf001/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
index 4ffccd4..6c54822 100644
--- a/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java
+++ b/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java
@@ -16,11 +16,16 @@
  */
 package org.apache.solr.util.stats;
 
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.SortedSet;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
 
 import com.codahale.metrics.Counter;
 import com.codahale.metrics.Gauge;
@@ -32,6 +37,7 @@ 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.SolrInputDocument;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.metrics.AggregateMetric;
 
@@ -40,6 +46,9 @@ import org.apache.solr.metrics.AggregateMetric;
  */
 public class MetricUtils {
 
+  public static final String NAME = "name";
+  public static final String VALUES = "values";
+
   static final String MS = "_ms";
 
   static final String MIN = "min";
@@ -61,9 +70,6 @@ public class MetricUtils {
   static final String P999 = "p999";
   static final String P999_MS = P999 + MS;
 
-  static final String VALUES = "values";
-
-
   /**
    * Adds metrics from a Timer to a NamedList, using well-known back-compat names.
    * @param lst The NamedList to add the metrics data to
@@ -103,61 +109,127 @@ public class MetricUtils {
    * @param mustMatchFilter a {@link MetricFilter}.
    *                        A metric <em>must</em> match this filter to be included in the output.
    * @param skipHistograms discard any {@link Histogram}-s and histogram parts of {@link Timer}-s.
+   * @param metadata optional metadata. If not null and not empty then this map will be added under a
+   *                 {@code _metadata_} key.
    * @return a {@link NamedList}
    */
   public static NamedList toNamedList(MetricRegistry registry, List<MetricFilter> shouldMatchFilters,
-                                      MetricFilter mustMatchFilter, boolean skipHistograms) {
-    NamedList response = new NamedList();
+                                      MetricFilter mustMatchFilter, boolean skipHistograms,
+                                      Map<String, Object> metadata) {
+    NamedList result = new NamedList();
+    toNamedMaps(registry, shouldMatchFilters, mustMatchFilter, skipHistograms, (k, v) -> {
+      result.add(k, new NamedList(v));
+    });
+    if (metadata != null && !metadata.isEmpty()) {
+      result.add("_metadata_", new NamedList(metadata));
+    }
+    return result;
+  }
+
+  /**
+   * Returns a representation of the given metric registry as a list of {@link SolrInputDocument}-s.
+   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 shouldMatchFilters a list of {@link MetricFilter} instances.
+   *                           A metric must match <em>any one</em> of the filters from this list to be
+   *                           included in the output
+   * @param mustMatchFilter a {@link MetricFilter}.
+   *                        A metric <em>must</em> match this filter to be included in the output.
+   * @param skipHistograms discard any {@link Histogram}-s and histogram parts of {@link Timer}-s.
+   * @param metadata optional metadata. If not null and not empty then this map will be added under a
+   *                 {@code _metadata_} key.
+   * @return a list of {@link SolrInputDocument}-s
+   */
+  public static List<SolrInputDocument> toSolrInputDocuments(MetricRegistry registry, List<MetricFilter> shouldMatchFilters,
+                                                             MetricFilter mustMatchFilter, boolean skipHistograms,
+                                                             Map<String, Object> metadata) {
+    List<SolrInputDocument> result = new LinkedList<>();
+    toSolrInputDocuments(registry, shouldMatchFilters, mustMatchFilter, skipHistograms, metadata, doc -> {
+      result.add(doc);
+    });
+    return result;
+  }
+
+  public static void toSolrInputDocuments(MetricRegistry registry, List<MetricFilter> shouldMatchFilters,
+                                          MetricFilter mustMatchFilter, boolean skipHistograms,
+                                          Map<String, Object> metadata, Consumer<SolrInputDocument> consumer) {
+    boolean addMetadata = metadata != null && !metadata.isEmpty();
+    toNamedMaps(registry, shouldMatchFilters, mustMatchFilter, skipHistograms, (k, v) -> {
+      SolrInputDocument doc = new SolrInputDocument();
+      doc.setField(NAME, k);
+      toSolrInputDocument(null, doc, v);
+      if (addMetadata) {
+        toSolrInputDocument(null, doc, metadata);
+      }
+      consumer.accept(doc);
+    });
+  }
+
+  public static void toSolrInputDocument(String prefix, SolrInputDocument doc, Map<String, Object> map) {
+    for (Map.Entry<String, Object> entry : map.entrySet()) {
+      if (entry.getValue() instanceof Map) { // flatten recursively
+        toSolrInputDocument(entry.getKey(), doc, (Map<String, Object>)entry.getValue());
+      } else {
+        String key = prefix != null ? prefix + "." + entry.getKey() : entry.getKey();
+        doc.addField(key, entry.getValue());
+      }
+    }
+  }
+
+  public static void toNamedMaps(MetricRegistry registry, List<MetricFilter> shouldMatchFilters,
+                MetricFilter mustMatchFilter, boolean skipHistograms,
+                BiConsumer<String, Map<String, Object>> consumer) {
     Map<String, Metric> metrics = registry.getMetrics();
     SortedSet<String> names = registry.getNames();
     names.stream()
         .filter(s -> shouldMatchFilters.stream().anyMatch(metricFilter -> metricFilter.matches(s, metrics.get(s))))
         .filter(s -> mustMatchFilter.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, skipHistograms));
-      } else if (metric instanceof Histogram) {
-        if (!skipHistograms) {
-          Histogram histogram = (Histogram) metric;
-          response.add(n, histogramToNamedList(histogram));
-        }
-      } else if (metric instanceof AggregateMetric) {
-        response.add(n, aggregateMetricToNamedList((AggregateMetric)metric));
-      }
-    });
-    return response;
+          Metric metric = metrics.get(n);
+          if (metric instanceof Counter) {
+            Counter counter = (Counter) metric;
+            consumer.accept(n, counterToMap(counter));
+          } else if (metric instanceof Gauge) {
+            Gauge gauge = (Gauge) metric;
+            consumer.accept(n, gaugeToMap(gauge));
+          } else if (metric instanceof Meter) {
+            Meter meter = (Meter) metric;
+            consumer.accept(n, meterToMap(meter));
+          } else if (metric instanceof Timer) {
+            Timer timer = (Timer) metric;
+            consumer.accept(n, timerToMap(timer, skipHistograms));
+          } else if (metric instanceof Histogram) {
+            if (!skipHistograms) {
+              Histogram histogram = (Histogram) metric;
+              consumer.accept(n, histogramToMap(histogram));
+            }
+          } else if (metric instanceof AggregateMetric) {
+            consumer.accept(n, aggregateMetricToMap((AggregateMetric)metric));
+          }
+        });
   }
 
-  static NamedList aggregateMetricToNamedList(AggregateMetric metric) {
-    NamedList response = new NamedList();
-    response.add("count", metric.size());
-    response.add(MAX, metric.getMax());
-    response.add(MIN, metric.getMin());
-    response.add(MEAN, metric.getMean());
-    response.add(STDDEV, metric.getStdDev());
+  static Map<String, Object> aggregateMetricToMap(AggregateMetric metric) {
+    Map<String, Object> response = new LinkedHashMap<>();
+    response.put("count", metric.size());
+    response.put(MAX, metric.getMax());
+    response.put(MIN, metric.getMin());
+    response.put(MEAN, metric.getMean());
+    response.put(STDDEV, metric.getStdDev());
     if (!metric.isEmpty()) {
-      NamedList values = new NamedList();
-      response.add(VALUES, values);
-      metric.getValues().forEach((k, v) -> values.add(k, v));
+      Map<String, Object> values = new LinkedHashMap<>();
+      response.put(VALUES, values);
+      metric.getValues().forEach((k, v) -> values.put(k, v));
     }
     return response;
   }
 
-  static NamedList histogramToNamedList(Histogram histogram) {
-    NamedList response = new NamedList();
+  static Map<String, Object> histogramToMap(Histogram histogram) {
+    Map<String, Object> response = new LinkedHashMap<>();
     Snapshot snapshot = histogram.getSnapshot();
-    response.add("count", histogram.getCount());
+    response.put("count", histogram.getCount());
     // non-time based values
     addSnapshot(response, snapshot, false);
     return response;
@@ -173,25 +245,25 @@ public class MetricUtils {
   }
 
   // some snapshots represent time in ns, other snapshots represent raw values (eg. chunk size)
-  static void addSnapshot(NamedList response, Snapshot snapshot, boolean ms) {
-    response.add((ms ? MIN_MS: MIN), nsToMs(ms, snapshot.getMin()));
-    response.add((ms ? MAX_MS: MAX), nsToMs(ms, snapshot.getMax()));
-    response.add((ms ? MEAN_MS : MEAN), nsToMs(ms, snapshot.getMean()));
-    response.add((ms ? MEDIAN_MS: MEDIAN), nsToMs(ms, snapshot.getMedian()));
-    response.add((ms ? STDDEV_MS: STDDEV), nsToMs(ms, snapshot.getStdDev()));
-    response.add((ms ? P75_MS: P75), nsToMs(ms, snapshot.get75thPercentile()));
-    response.add((ms ? P95_MS: P95), nsToMs(ms, snapshot.get95thPercentile()));
-    response.add((ms ? P99_MS: P99), nsToMs(ms, snapshot.get99thPercentile()));
-    response.add((ms ? P999_MS: P999), nsToMs(ms, snapshot.get999thPercentile()));
-  }
-
-  static NamedList timerToNamedList(Timer timer, boolean skipHistograms) {
-    NamedList response = new NamedList();
-    response.add("count", timer.getCount());
-    response.add("meanRate", timer.getMeanRate());
-    response.add("1minRate", timer.getOneMinuteRate());
-    response.add("5minRate", timer.getFiveMinuteRate());
-    response.add("15minRate", timer.getFifteenMinuteRate());
+  static void addSnapshot(Map<String, Object> response, Snapshot snapshot, boolean ms) {
+    response.put((ms ? MIN_MS: MIN), nsToMs(ms, snapshot.getMin()));
+    response.put((ms ? MAX_MS: MAX), nsToMs(ms, snapshot.getMax()));
+    response.put((ms ? MEAN_MS : MEAN), nsToMs(ms, snapshot.getMean()));
+    response.put((ms ? MEDIAN_MS: MEDIAN), nsToMs(ms, snapshot.getMedian()));
+    response.put((ms ? STDDEV_MS: STDDEV), nsToMs(ms, snapshot.getStdDev()));
+    response.put((ms ? P75_MS: P75), nsToMs(ms, snapshot.get75thPercentile()));
+    response.put((ms ? P95_MS: P95), nsToMs(ms, snapshot.get95thPercentile()));
+    response.put((ms ? P99_MS: P99), nsToMs(ms, snapshot.get99thPercentile()));
+    response.put((ms ? P999_MS: P999), nsToMs(ms, snapshot.get999thPercentile()));
+  }
+
+  static Map<String,Object> timerToMap(Timer timer, boolean skipHistograms) {
+    Map<String, Object> response = new LinkedHashMap<>();
+    response.put("count", timer.getCount());
+    response.put("meanRate", timer.getMeanRate());
+    response.put("1minRate", timer.getOneMinuteRate());
+    response.put("5minRate", timer.getFiveMinuteRate());
+    response.put("15minRate", timer.getFifteenMinuteRate());
     if (!skipHistograms) {
       // time-based values in nanoseconds
       addSnapshot(response, timer.getSnapshot(), true);
@@ -199,25 +271,25 @@ public class MetricUtils {
     return response;
   }
 
-  static NamedList meterToNamedList(Meter meter) {
-    NamedList response = new NamedList();
-    response.add("count", meter.getCount());
-    response.add("meanRate", meter.getMeanRate());
-    response.add("1minRate", meter.getOneMinuteRate());
-    response.add("5minRate", meter.getFiveMinuteRate());
-    response.add("15minRate", meter.getFifteenMinuteRate());
+  static Map<String, Object> meterToMap(Meter meter) {
+    Map<String, Object> response = new LinkedHashMap<>();
+    response.put("count", meter.getCount());
+    response.put("meanRate", meter.getMeanRate());
+    response.put("1minRate", meter.getOneMinuteRate());
+    response.put("5minRate", meter.getFiveMinuteRate());
+    response.put("15minRate", meter.getFifteenMinuteRate());
     return response;
   }
 
-  static NamedList gaugeToNamedList(Gauge gauge) {
-    NamedList response = new NamedList();
-    response.add("value", gauge.getValue());
+  static Map<String, Object> gaugeToMap(Gauge gauge) {
+    Map<String, Object> response = new LinkedHashMap<>();
+    response.put("value", gauge.getValue());
     return response;
   }
 
-  static NamedList counterToNamedList(Counter counter) {
-    NamedList response = new NamedList();
-    response.add("count", counter.getCount());
+  static Map<String, Object> counterToMap(Counter counter) {
+    Map<String, Object> response = new LinkedHashMap<>();
+    response.put("count", counter.getCount());
     return response;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/197bf001/solr/core/src/test/org/apache/solr/metrics/reporters/solr/SolrReplicaReporterTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/reporters/solr/SolrReplicaReporterTest.java b/solr/core/src/test/org/apache/solr/metrics/reporters/solr/SolrReplicaReporterTest.java
index 0342786..55a59ba 100644
--- a/solr/core/src/test/org/apache/solr/metrics/reporters/solr/SolrReplicaReporterTest.java
+++ b/solr/core/src/test/org/apache/solr/metrics/reporters/solr/SolrReplicaReporterTest.java
@@ -14,6 +14,6 @@ public class SolrReplicaReporterTest extends AbstractFullDistribZkTestBase {
   @Test
   public void test() throws Exception {
     printLayout();
-    //Thread.sleep(10000000);
+    Thread.sleep(10000000);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/197bf001/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
index 4418318..0954569 100644
--- a/solr/core/src/test/org/apache/solr/util/stats/MetricUtilsTest.java
+++ b/solr/core/src/test/org/apache/solr/util/stats/MetricUtilsTest.java
@@ -36,7 +36,7 @@ public class MetricUtilsTest extends SolrTestCaseJ4 {
       timer.update(Math.abs(random().nextInt()) + 1, TimeUnit.NANOSECONDS);
     }
     // obtain timer metrics
-    NamedList lst = MetricUtils.timerToNamedList(timer, false);
+    NamedList lst = new NamedList(MetricUtils.timerToMap(timer, false));
     // check that expected metrics were obtained
     assertEquals(14, lst.size());
     final Snapshot snapshot = timer.getSnapshot();