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 2018/05/29 09:28:06 UTC

[2/3] lucene-solr:master: SOLR-11779: Basic long-term collection of aggregated metrics.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6bbce38b/solr/core/src/test/org/apache/solr/cloud/MetricsHistoryIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/MetricsHistoryIntegrationTest.java b/solr/core/src/test/org/apache/solr/cloud/MetricsHistoryIntegrationTest.java
new file mode 100644
index 0000000..2012a1a
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/MetricsHistoryIntegrationTest.java
@@ -0,0 +1,193 @@
+/*
+ * 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.cloud;
+
+import javax.imageio.ImageIO;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.cloud.SolrCloudManager;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.GenericSolrRequest;
+import org.apache.solr.cloud.autoscaling.sim.SimCloudManager;
+import org.apache.solr.common.params.CollectionAdminParams;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.Base64;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.util.LogLevel;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ *
+ */
+@LuceneTestCase.Slow
+@LogLevel("org.apache.solr.handler.admin=DEBUG")
+public class MetricsHistoryIntegrationTest extends SolrCloudTestCase {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static SolrCloudManager cloudManager;
+  private static SolrClient solrClient;
+  private static TimeSource timeSource;
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    boolean simulated = random().nextBoolean();
+    if (simulated) {
+      cloudManager = SimCloudManager.createCluster(1, TimeSource.get("simTime:50"));
+      solrClient = ((SimCloudManager)cloudManager).simGetSolrClient();
+    }
+    configureCluster(1)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
+    if (!simulated) {
+      cloudManager = cluster.getJettySolrRunner(0).getCoreContainer().getZkController().getSolrCloudManager();
+      solrClient = cluster.getSolrClient();
+    }
+    timeSource = cloudManager.getTimeSource();
+    // create .system
+    CollectionAdminRequest.createCollection(CollectionAdminParams.SYSTEM_COLL, null, 1, 1)
+        .process(solrClient);
+    CloudTestUtils.waitForState(cloudManager, CollectionAdminParams.SYSTEM_COLL,
+        30, TimeUnit.SECONDS, CloudTestUtils.clusterShape(1, 1));
+    solrClient.query(CollectionAdminParams.SYSTEM_COLL, params(CommonParams.Q, "*:*"));
+    // sleep a little to allow the handler to collect some metrics
+    timeSource.sleep(90000);
+  }
+
+  @AfterClass
+  public static void teardown() throws Exception {
+    if (cloudManager instanceof SimCloudManager) {
+      cloudManager.close();
+    }
+    solrClient = null;
+    cloudManager = null;
+  }
+
+  @Test
+  public void testList() throws Exception {
+    NamedList<Object> rsp = solrClient.request(createHistoryRequest(params(CommonParams.ACTION, "list")));
+    assertNotNull(rsp);
+    // expected solr.jvm, solr.node and solr.collection..system
+    List<String> lst = (List<String>)rsp.get("metrics");
+    assertNotNull(lst);
+    assertEquals(lst.toString(), 3, lst.size());
+    assertTrue(lst.toString(), lst.contains("solr.jvm"));
+    assertTrue(lst.toString(), lst.contains("solr.node"));
+    assertTrue(lst.toString(), lst.contains("solr.collection..system"));
+  }
+
+  @Test
+  public void testStatus() throws Exception {
+    NamedList<Object> rsp = solrClient.request(createHistoryRequest(
+        params(CommonParams.ACTION, "status", CommonParams.NAME, "solr.jvm")));
+    assertNotNull(rsp);
+    NamedList<Object> map = (NamedList<Object>)rsp.get("metrics");
+    assertEquals(map.toString(), 1, map.size());
+    map = (NamedList<Object>)map.get("solr.jvm");
+    assertNotNull(map);
+    NamedList<Object> status = (NamedList<Object>)map.get("status");
+    assertNotNull(status);
+    assertEquals(status.toString(), 7, status.size());
+    List<Object> lst = (List<Object>)status.get("datasources");
+    assertNotNull(lst);
+    assertEquals(lst.toString(), 3, lst.size());
+    lst = (List<Object>)status.get("archives");
+    assertNotNull(lst);
+    assertEquals(lst.toString(), 5, lst.size());
+  }
+
+  @Test
+  public void testGet() throws Exception {
+    NamedList<Object> rsp = solrClient.request(createHistoryRequest(params(
+        CommonParams.ACTION, "get", CommonParams.NAME, "solr.jvm")));
+    assertNotNull(rsp);
+    // default format is LIST
+    NamedList<Object> data = (NamedList<Object>)rsp.findRecursive("metrics", "solr.jvm", "data");
+    assertNotNull(data);
+    data.forEach((k, v) -> {
+      NamedList<Object> entry = (NamedList<Object>)v;
+      List<Object> lst = entry.getAll("timestamps");
+      assertNotNull(lst);
+      assertTrue("timestamps", lst.size() > 0);
+      // 3 metrics, so the total size of values is 3 * the size of timestamps
+      entry = (NamedList<Object>)entry.get("values");
+      assertNotNull(entry);
+      assertEquals(lst.size() * 3, entry.size());
+    });
+
+    // get STRING
+    rsp = solrClient.request(createHistoryRequest(params(
+        CommonParams.ACTION, "get", CommonParams.NAME, "solr.jvm", "format", "string")));
+    data = (NamedList<Object>)rsp.findRecursive("metrics", "solr.jvm", "data");
+    assertNotNull(data);
+    data.forEach((k, v) -> {
+      NamedList<Object> entry = (NamedList<Object>)v;
+      List<Object> lst = entry.getAll("timestamps");
+      assertNotNull(lst);
+      assertEquals("timestamps", 1, lst.size());
+      String timestampString = (String)lst.get(0);
+      String[] timestamps = timestampString.split(("\n"));
+      assertTrue(timestampString, timestamps.length > 1);
+      entry = (NamedList<Object>)entry.get("values");
+      assertNotNull(entry);
+      assertEquals(3, entry.size());
+      entry.forEach((vk, vv) -> {
+        String valString = (String)vv;
+        String[] values = valString.split("\n");
+        assertEquals(valString, timestamps.length, values.length);
+      });
+    });
+
+    // get GRAPH
+    rsp = solrClient.request(createHistoryRequest(params(
+        CommonParams.ACTION, "get", CommonParams.NAME, "solr.jvm", "format", "graph")));
+    data = (NamedList<Object>)rsp.findRecursive("metrics", "solr.jvm", "data");
+    assertNotNull(data);
+    data.forEach((k, v) -> {
+      NamedList<Object> entry = (NamedList<Object>) v;
+      entry = (NamedList<Object>)entry.get("values");
+      assertNotNull(entry);
+      assertEquals(3, entry.size());
+      entry.forEach((vk, vv) -> {
+        String valString = (String)vv;
+        byte[] img = Base64.base64ToByteArray(valString);
+        try {
+          ImageIO.read(new ByteArrayInputStream(img));
+        } catch (IOException e) {
+          fail("Error reading image data: " + e.toString());
+        }
+      });
+    });
+  }
+
+  public static SolrRequest createHistoryRequest(SolrParams params) {
+    return new GenericSolrRequest(SolrRequest.METHOD.GET, "/admin/metrics/history", params);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6bbce38b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/LiveNodesSet.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/LiveNodesSet.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/LiveNodesSet.java
index ca4ed71..3f5d5f4 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/LiveNodesSet.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/LiveNodesSet.java
@@ -18,6 +18,7 @@ package org.apache.solr.cloud.autoscaling.sim;
 
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Iterator;
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
@@ -28,7 +29,7 @@ import org.apache.solr.common.cloud.LiveNodesListener;
 /**
  * This class represents a set of live nodes and allows adding listeners to track their state.
  */
-public class LiveNodesSet {
+public class LiveNodesSet implements Iterable<String> {
 
   private final Set<String> set = ConcurrentHashMap.newKeySet();
   private final Set<LiveNodesListener> listeners = ConcurrentHashMap.newKeySet();
@@ -100,4 +101,9 @@ public class LiveNodesSet {
     set.clear();
     fireListeners(oldNodes, Collections.emptySortedSet());
   }
+
+  @Override
+  public Iterator<String> iterator() {
+    return set.iterator();
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6bbce38b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java
index 9641552..c09d4a4 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java
@@ -18,6 +18,7 @@
 package org.apache.solr.cloud.autoscaling.sim;
 
 import java.io.ByteArrayOutputStream;
+import java.io.File;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
@@ -36,6 +37,10 @@ import java.util.concurrent.Future;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
+import com.codahale.metrics.jvm.ClassLoadingGaugeSet;
+import com.codahale.metrics.jvm.GarbageCollectorMetricSet;
+import com.codahale.metrics.jvm.MemoryUsageGaugeSet;
+import com.codahale.metrics.jvm.ThreadStatesGaugeSet;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrResponse;
@@ -79,10 +84,18 @@ import org.apache.solr.common.util.ObjectCache;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.CloudConfig;
+import org.apache.solr.core.SolrInfoBean;
 import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.handler.admin.MetricsHandler;
+import org.apache.solr.handler.admin.MetricsHistoryHandler;
+import org.apache.solr.metrics.AltBufferPoolMetricSet;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.OperatingSystemMetricSet;
+import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.util.DefaultSolrThreadFactory;
+import org.apache.solr.util.MockSearchableSolrClient;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -101,9 +114,11 @@ public class SimCloudManager implements SolrCloudManager {
   private final LiveNodesSet liveNodesSet = new LiveNodesSet();
   private final DistributedQueueFactory queueFactory;
   private final ObjectCache objectCache = new ObjectCache();
-  private TimeSource timeSource;
+  private final SolrMetricManager metricManager = new SolrMetricManager();
+  private final String metricTag;
 
   private final List<SolrInputDocument> systemColl = Collections.synchronizedList(new ArrayList<>());
+  private final MockSearchableSolrClient solrClient;
   private final Map<String, AtomicLong> opCounts = new ConcurrentSkipListMap<>();
 
 
@@ -111,6 +126,8 @@ public class SimCloudManager implements SolrCloudManager {
   private Overseer.OverseerThread triggerThread;
   private ThreadGroup triggerThreadGroup;
   private SolrResourceLoader loader;
+  private MetricsHistoryHandler historyHandler;
+  private TimeSource timeSource;
 
   private static int nodeIdPort = 10000;
   public static int DEFAULT_DISK = 1000; // 1000 GB
@@ -142,13 +159,72 @@ public class SimCloudManager implements SolrCloudManager {
     stateManager.makePath(ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH);
     stateManager.makePath(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH);
     stateManager.makePath(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH);
+    stateManager.makePath(Overseer.OVERSEER_ELECT);
+
+    // register common metrics
+    metricTag = Integer.toHexString(hashCode());
+    String registryName = SolrMetricManager.getRegistryName(SolrInfoBean.Group.jvm);
+    metricManager.registerAll(registryName, new AltBufferPoolMetricSet(), true, "buffers");
+    metricManager.registerAll(registryName, new ClassLoadingGaugeSet(), true, "classes");
+    metricManager.registerAll(registryName, new OperatingSystemMetricSet(), true, "os");
+    metricManager.registerAll(registryName, new GarbageCollectorMetricSet(), true, "gc");
+    metricManager.registerAll(registryName, new MemoryUsageGaugeSet(), true, "memory");
+    metricManager.registerAll(registryName, new ThreadStatesGaugeSet(), true, "threads"); // todo should we use CachedThreadStatesGaugeSet instead?
+    MetricsMap sysprops = new MetricsMap((detailed, map) -> {
+      System.getProperties().forEach((k, v) -> {
+        map.put(String.valueOf(k), v);
+      });
+    });
+    metricManager.registerGauge(null, registryName, sysprops, metricTag, true, "properties", "system");
+
+    registryName = SolrMetricManager.getRegistryName(SolrInfoBean.Group.node);
+    metricManager.registerGauge(null, registryName, () -> new File("/").getUsableSpace(),
+        metricTag, true, "usableSpace", SolrInfoBean.Category.CONTAINER.toString(), "fs", "coreRoot");
+
+    solrClient = new MockSearchableSolrClient() {
+      @Override
+      public NamedList<Object> request(SolrRequest request, String collection) throws SolrServerException, IOException {
+        if (collection != null) {
+          if (request instanceof AbstractUpdateRequest) {
+            ((AbstractUpdateRequest)request).setParam("collection", collection);
+          } else if (request instanceof QueryRequest) {
+            if (request.getPath() != null && (
+                request.getPath().startsWith("/admin/autoscaling") ||
+                request.getPath().startsWith("/cluster/autoscaling") ||
+            request.getPath().startsWith("/admin/metrics/history") ||
+                request.getPath().startsWith("/cluster/metrics/history")
+            )) {
+              // forward it
+              ModifiableSolrParams params = new ModifiableSolrParams(request.getParams());
+              params.set("collection", collection);
+              request = new QueryRequest(params);
+            } else {
+              // search request
+              return super.request(request, collection);
+            }
+          } else {
+            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "when collection != null only UpdateRequest and QueryRequest are supported: request=" + request + ", collection=" + collection);
+          }
+        }
+        try {
+          SolrResponse rsp = SimCloudManager.this.request(request);
+          return rsp.getResponse();
+        } catch (UnsupportedOperationException e) {
+          throw new SolrServerException(e);
+        }
+      }
+    };
+
 
     this.timeSource = timeSource != null ? timeSource : TimeSource.NANO_TIME;
     this.clusterStateProvider = new SimClusterStateProvider(liveNodesSet, this);
     this.nodeStateProvider = new SimNodeStateProvider(liveNodesSet, this.stateManager, this.clusterStateProvider, null);
     this.queueFactory = new GenericDistributedQueueFactory(stateManager);
     this.simCloudManagerPool = ExecutorUtil.newMDCAwareFixedThreadPool(200, new DefaultSolrThreadFactory("simCloudManagerPool"));
+
     this.autoScalingHandler = new AutoScalingHandler(this, loader);
+
+
     triggerThreadGroup = new ThreadGroup("Simulated Overseer autoscaling triggers");
     OverseerTriggerThread trigger = new OverseerTriggerThread(loader, this,
         new CloudConfig.CloudConfigBuilder("nonexistent", 0, "sim").build());
@@ -167,13 +243,7 @@ public class SimCloudManager implements SolrCloudManager {
   public static SimCloudManager createCluster(int numNodes, TimeSource timeSource) throws Exception {
     SimCloudManager cloudManager = new SimCloudManager(timeSource);
     for (int i = 1; i <= numNodes; i++) {
-      Map<String, Object> values = createNodeValues(null);
-//      if (i == 1) { // designated Overseer ?
-        //values.put(ImplicitSnitch.NODEROLE, "overseer");
-//      }
-      String nodeId = (String)values.get(ImplicitSnitch.NODE);
-      cloudManager.getSimClusterStateProvider().simAddNode(nodeId);
-      cloudManager.getSimNodeStateProvider().simSetNodeValues(nodeId, values);
+      cloudManager.simAddNode();
     }
     return cloudManager;
   }
@@ -325,6 +395,12 @@ public class SimCloudManager implements SolrCloudManager {
     clusterStateProvider.simAddNode(nodeId);
     nodeStateProvider.simSetNodeValues(nodeId, values);
     LOG.trace("-- added node " + nodeId);
+    // initialize history handler if this is the first node
+    if (historyHandler == null && liveNodesSet.size() == 1) {
+      MetricsHandler metricsHandler = new MetricsHandler(metricManager);
+      historyHandler = new MetricsHistoryHandler(nodeId, metricsHandler, solrClient, this, Collections.emptyMap());
+      historyHandler.initializeMetrics(metricManager, SolrMetricManager.getRegistryName(SolrInfoBean.Group.node), metricTag, CommonParams.METRICS_HISTORY_PATH);
+    }
     return nodeId;
   }
 
@@ -397,29 +473,30 @@ public class SimCloudManager implements SolrCloudManager {
    * @return simulated SolrClient.
    */
   public SolrClient simGetSolrClient() {
-    return new SolrClient() {
-      @Override
-      public NamedList<Object> request(SolrRequest request, String collection) throws SolrServerException, IOException {
-        if (collection != null) {
-          if (request instanceof AbstractUpdateRequest) {
-            ((AbstractUpdateRequest)request).setParam("collection", collection);
-          } else if (request instanceof QueryRequest) {
-            ModifiableSolrParams params = new ModifiableSolrParams(request.getParams());
-            params.set("collection", collection);
-            request = new QueryRequest(params);
-          } else {
-            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "when collection != null only UpdateRequest and QueryRequest are supported: request=" + request + ", collection=" + collection);
-          }
-        }
-        SolrResponse rsp = SimCloudManager.this.request(request);
-        return rsp.getResponse();
-      }
-
-      @Override
-      public void close() throws IOException {
-
-      }
-    };
+    return solrClient;
+//    return new SolrClient() {
+//      @Override
+//      public NamedList<Object> request(SolrRequest request, String collection) throws SolrServerException, IOException {
+//        if (collection != null) {
+//          if (request instanceof AbstractUpdateRequest) {
+//            ((AbstractUpdateRequest)request).setParam("collection", collection);
+//          } else if (request instanceof QueryRequest) {
+//            ModifiableSolrParams params = new ModifiableSolrParams(request.getParams());
+//            params.set("collection", collection);
+//            request = new QueryRequest(params);
+//          } else {
+//            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "when collection != null only UpdateRequest and QueryRequest are supported: request=" + request + ", collection=" + collection);
+//          }
+//        }
+//        SolrResponse rsp = SimCloudManager.this.request(request);
+//        return rsp.getResponse();
+//      }
+//
+//      @Override
+//      public void close() throws IOException {
+//
+//      }
+//    };
   }
 
   /**
@@ -498,6 +575,10 @@ public class SimCloudManager implements SolrCloudManager {
     return count != null ? count.get() : 0L;
   }
 
+  public SolrMetricManager getMetricManager() {
+    return metricManager;
+  }
+
   // --------- interface methods -----------
 
 
@@ -558,50 +639,71 @@ public class SimCloudManager implements SolrCloudManager {
 
     LOG.trace("--- got SolrRequest: " + req.getMethod() + " " + req.getPath() +
         (req.getParams() != null ? " " + req.getParams().toQueryString() : ""));
-    if (req.getPath() != null && req.getPath().startsWith("/admin/autoscaling") ||
-        req.getPath().startsWith("/cluster/autoscaling")) {
-      incrementCount("autoscaling");
-      ModifiableSolrParams params = new ModifiableSolrParams(req.getParams());
-      params.set(CommonParams.PATH, req.getPath());
-      LocalSolrQueryRequest queryRequest = new LocalSolrQueryRequest(null, params);
-      RequestWriter.ContentWriter cw = req.getContentWriter("application/json");
-      ByteArrayOutputStream baos = new ByteArrayOutputStream();
-      cw.write(baos);
-      String payload = baos.toString("UTF-8");
-      LOG.trace("-- payload: {}", payload);
-      queryRequest.setContentStreams(Collections.singletonList(new ContentStreamBase.StringStream(payload)));
-      queryRequest.getContext().put("httpMethod", req.getMethod().toString());
-      SolrQueryResponse queryResponse = new SolrQueryResponse();
-      autoScalingHandler.handleRequest(queryRequest, queryResponse);
-      if (queryResponse.getException() != null) {
-        LOG.debug("-- exception handling request", queryResponse.getException());
-        throw new IOException(queryResponse.getException());
+    if (req.getPath() != null) {
+      if (req.getPath().startsWith("/admin/autoscaling") ||
+          req.getPath().startsWith("/cluster/autoscaling") ||
+          req.getPath().startsWith("/admin/metrics/history") ||
+          req.getPath().startsWith("/cluster/metrics/history")
+          ) {
+        metricManager.registry("solr.node").counter("ADMIN." + req.getPath() + ".requests").inc();
+        boolean autoscaling = req.getPath().contains("autoscaling");
+        if (autoscaling) {
+          incrementCount("autoscaling");
+        } else {
+          incrementCount("metricsHistory");
+        }
+        ModifiableSolrParams params = new ModifiableSolrParams(req.getParams());
+        params.set(CommonParams.PATH, req.getPath());
+        LocalSolrQueryRequest queryRequest = new LocalSolrQueryRequest(null, params);
+        if (autoscaling) {
+          RequestWriter.ContentWriter cw = req.getContentWriter("application/json");
+          ByteArrayOutputStream baos = new ByteArrayOutputStream();
+          cw.write(baos);
+          String payload = baos.toString("UTF-8");
+          LOG.trace("-- payload: {}", payload);
+          queryRequest.setContentStreams(Collections.singletonList(new ContentStreamBase.StringStream(payload)));
+        }
+        queryRequest.getContext().put("httpMethod", req.getMethod().toString());
+        SolrQueryResponse queryResponse = new SolrQueryResponse();
+        if (autoscaling) {
+          autoScalingHandler.handleRequest(queryRequest, queryResponse);
+        } else {
+          if (historyHandler != null) {
+            historyHandler.handleRequest(queryRequest, queryResponse);
+          } else {
+            throw new UnsupportedOperationException("must add at least 1 node first");
+          }
+        }
+        if (queryResponse.getException() != null) {
+          LOG.debug("-- exception handling request", queryResponse.getException());
+          throw new IOException(queryResponse.getException());
+        }
+        SolrResponse rsp = new SolrResponseBase();
+        rsp.setResponse(queryResponse.getValues());
+        LOG.trace("-- response: {}", rsp);
+        return rsp;
       }
-      SolrResponse rsp = new SolrResponseBase();
-      rsp.setResponse(queryResponse.getValues());
-      LOG.trace("-- response: {}", rsp);
-      return rsp;
     }
     if (req instanceof UpdateRequest) {
       incrementCount("update");
-      // support only updates to the system collection
       UpdateRequest ureq = (UpdateRequest)req;
       String collection = ureq.getCollection();
-      if (collection != null && !collection.equals(CollectionAdminParams.SYSTEM_COLL)) {
-        // simulate an update
-        return clusterStateProvider.simUpdate(ureq);
-      } else {
+      UpdateResponse rsp = clusterStateProvider.simUpdate(ureq);
+      if (collection == null || collection.equals(CollectionAdminParams.SYSTEM_COLL)) {
         List<SolrInputDocument> docs = ureq.getDocuments();
         if (docs != null) {
           systemColl.addAll(docs);
         }
         return new UpdateResponse();
+      } else {
+        return rsp;
       }
     }
     // support only a specific subset of collection admin ops
     if (!(req instanceof CollectionAdminRequest)) {
       throw new UnsupportedOperationException("Only some CollectionAdminRequest-s are supported: " + req.getClass().getName());
     }
+    metricManager.registry("solr.node").counter("ADMIN." + req.getPath() + ".requests").inc();
     SolrParams params = req.getParams();
     String a = params.get(CoreAdminParams.ACTION);
     SolrResponse rsp = new SolrResponseBase();
@@ -721,6 +823,9 @@ public class SimCloudManager implements SolrCloudManager {
 
   @Override
   public void close() throws IOException {
+    if (historyHandler != null) {
+      IOUtils.closeQuietly(historyHandler);
+    }
     IOUtils.closeQuietly(clusterStateProvider);
     IOUtils.closeQuietly(nodeStateProvider);
     IOUtils.closeQuietly(stateManager);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6bbce38b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
index f2feb8e..ca2dd48 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
@@ -29,6 +29,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
+import java.util.NoSuchElementException;
 import java.util.Random;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
@@ -50,10 +51,12 @@ import org.apache.solr.client.solrj.impl.ClusterStateProvider;
 import org.apache.solr.client.solrj.request.UpdateRequest;
 import org.apache.solr.client.solrj.response.UpdateResponse;
 import org.apache.solr.cloud.ActionThrottle;
+import org.apache.solr.cloud.Overseer;
 import org.apache.solr.cloud.api.collections.AddReplicaCmd;
 import org.apache.solr.cloud.api.collections.Assign;
 import org.apache.solr.cloud.api.collections.CreateCollectionCmd;
 import org.apache.solr.cloud.api.collections.CreateShardCmd;
+import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler;
 import org.apache.solr.cloud.api.collections.SplitShardCmd;
 import org.apache.solr.cloud.overseer.ClusterStateMutator;
 import org.apache.solr.cloud.overseer.CollectionMutator;
@@ -69,11 +72,14 @@ import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.cloud.rule.ImplicitSnitch;
+import org.apache.solr.common.params.CollectionAdminParams;
 import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.params.CommonAdminParams;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Utils;
+import org.apache.solr.core.SolrInfoBean;
+import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.zookeeper.CreateMode;
 import org.junit.Assert;
 import org.slf4j.Logger;
@@ -125,6 +131,8 @@ public class SimClusterStateProvider implements ClusterStateProvider {
 
 
   private volatile int clusterStateVersion = 0;
+  private volatile String overseerLeader = null;
+
   private Map<String, Object> lastSavedProperties = null;
 
   private AtomicReference<Map<String, DocCollection>> collectionsStatesRef = new AtomicReference<>();
@@ -234,6 +242,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
     }
     liveNodes.add(nodeId);
     createEphemeralLiveNode(nodeId);
+    updateOverseerLeader();
     nodeReplicaMap.putIfAbsent(nodeId, new ArrayList<>());
   }
 
@@ -255,6 +264,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
       }
       // remove ephemeral nodes
       stateManager.getRoot().removeEphemeralChildren(nodeId);
+      updateOverseerLeader();
       // create a nodeLost marker if needed
       AutoScalingConfig cfg = stateManager.getAutoScalingConfig(null);
       if (cfg.hasTriggerForEvents(TriggerEventType.NODELOST)) {
@@ -269,6 +279,35 @@ public class SimClusterStateProvider implements ClusterStateProvider {
     }
   }
 
+  private synchronized void updateOverseerLeader() throws Exception {
+    if (overseerLeader != null && liveNodes.contains(overseerLeader)) {
+      return;
+    }
+    String path = Overseer.OVERSEER_ELECT + "/leader";
+    if (liveNodes.isEmpty()) {
+      overseerLeader = null;
+      // remove it from ZK
+      try {
+        cloudManager.getDistribStateManager().removeData(path, -1);
+      } catch (NoSuchElementException e) {
+        // ignore
+      }
+      return;
+    }
+    // pick first
+    overseerLeader = liveNodes.iterator().next();
+    LOG.debug("--- new Overseer leader: " + overseerLeader);
+    // record it in ZK
+    Map<String, Object> id = new HashMap<>();
+    id.put("id", cloudManager.getTimeSource().getTimeNs() +
+        "-" + overseerLeader + "-n_0000000000");
+    try {
+      cloudManager.getDistribStateManager().makePath(path, Utils.toJSON(id), CreateMode.EPHEMERAL, false);
+    } catch (Exception e) {
+      LOG.warn("Exception saving overseer leader id", e);
+    }
+  }
+
   // this method needs to be called under a lock
   private void setReplicaStates(String nodeId, Replica.State state, Set<String> changedCollections) {
     List<ReplicaInfo> replicas = nodeReplicaMap.get(nodeId);
@@ -423,6 +462,13 @@ public class SimClusterStateProvider implements ClusterStateProvider {
         disk = SimCloudManager.DEFAULT_DISK;
       }
       cloudManager.getSimNodeStateProvider().simSetNodeValue(nodeId, ImplicitSnitch.DISK, disk - 1);
+      // fake metrics
+      String registry = SolrMetricManager.getRegistryName(SolrInfoBean.Group.core, replicaInfo.getCollection(),
+          replicaInfo.getShard(),
+          Utils.parseMetricsReplicaName(replicaInfo.getCollection(), replicaInfo.getCore()));
+      cloudManager.getMetricManager().registry(registry).counter("UPDATE./update.requests");
+      cloudManager.getMetricManager().registry(registry).counter("QUERY./select.requests");
+      cloudManager.getMetricManager().registerGauge(null, registry, () -> 1000, "", true, "INDEX.sizeInBytes");
       if (runLeaderElection) {
         simRunLeaderElection(Collections.singleton(replicaInfo.getCollection()), true);
       }
@@ -1011,6 +1057,22 @@ public class SimClusterStateProvider implements ClusterStateProvider {
     }
   }
 
+  public synchronized void createSystemCollection() throws IOException {
+    try {
+      if (simListCollections().contains(CollectionAdminParams.SYSTEM_COLL)) {
+        return;
+      }
+      ZkNodeProps props = new ZkNodeProps(
+          NAME, CollectionAdminParams.SYSTEM_COLL,
+          REPLICATION_FACTOR, "1",
+          OverseerCollectionMessageHandler.NUM_SLICES, "1"
+      );
+      simCreateCollection(props, new NamedList());
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+  }
+
   /**
    * Simulate an update by modifying replica metrics.
    * The following core metrics are updated:
@@ -1035,7 +1097,12 @@ public class SimClusterStateProvider implements ClusterStateProvider {
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Collection not set");
     }
     if (!simListCollections().contains(collection)) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Collection '" + collection + "' doesn't exist");
+      if (CollectionAdminParams.SYSTEM_COLL.equals(collection)) {
+        // auto-create
+        createSystemCollection();
+      } else {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Collection '" + collection + "' doesn't exist");
+      }
     }
     // always reset first to get the current metrics - it's easier than to keep matching
     // Replica with ReplicaInfo where the current real counts are stored
@@ -1057,6 +1124,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
             LOG.debug("-- no leader in " + s);
             continue;
           }
+          cloudManager.getMetricManager().registry(createRegistryName(collection, s.getName(), leader)).counter("UPDATE./update.requests").inc();
           String numDocsStr = leader.getStr("SEARCHER.searcher.numDocs");
           if (numDocsStr == null) {
             LOG.debug("-- no docs in " + leader);
@@ -1085,7 +1153,14 @@ public class SimClusterStateProvider implements ClusterStateProvider {
             throw new UnsupportedOperationException("Only '*:*' query is supported in deleteByQuery");
           }
           for (Slice s : coll.getSlices()) {
-            String numDocsStr = s.getLeader().getStr("SEARCHER.searcher.numDocs");
+            Replica leader = s.getLeader();
+            if (leader == null) {
+              LOG.debug("-- no leader in " + s);
+              continue;
+            }
+
+            cloudManager.getMetricManager().registry(createRegistryName(collection, s.getName(), leader)).counter("UPDATE./update.requests").inc();
+            String numDocsStr = leader.getStr("SEARCHER.searcher.numDocs");
             if (numDocsStr == null) {
               continue;
             }
@@ -1111,6 +1186,10 @@ public class SimClusterStateProvider implements ClusterStateProvider {
             throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Document without id: " + doc);
           }
           Slice s = router.getTargetSlice(id, null, null, req.getParams(), coll);
+          Replica leader = s.getLeader();
+          if (leader != null) {
+            cloudManager.getMetricManager().registry(createRegistryName(collection, s.getName(), leader)).counter("UPDATE./update.requests").inc();
+          }
           modified = true;
           try {
             simSetShardValue(collection, s.getName(), "SEARCHER.searcher.numDocs", 1, true, false);
@@ -1132,6 +1211,11 @@ public class SimClusterStateProvider implements ClusterStateProvider {
     return new UpdateResponse();
   }
 
+  private static String createRegistryName(String collection, String shard, Replica r) {
+    return SolrMetricManager.getRegistryName(SolrInfoBean.Group.core, collection, shard,
+        Utils.parseMetricsReplicaName(collection, r.getCoreName()));
+  }
+
   /**
    * Saves cluster properties to clusterprops.json.
    * @return current properties

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6bbce38b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestComputePlanAction.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestComputePlanAction.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestComputePlanAction.java
index cdc4173..6f47b7f 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestComputePlanAction.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestComputePlanAction.java
@@ -273,7 +273,7 @@ public class TestComputePlanAction extends SimSolrCloudTestCase {
     String setClusterPolicyCommand = "{" +
         " 'set-cluster-policy': [" +
         "      {'cores':'<10', 'node':'#ANY'}," +
-        "      {'replica':'<3', 'shard': '#EACH', 'node': '#ANY'}," +
+        "      {'replica':'<5', 'shard': '#EACH', 'node': '#ANY'}," +
         "      {'nodeRole':'overseer', 'replica':0}" +
         "    ]" +
         "}";
@@ -282,7 +282,7 @@ public class TestComputePlanAction extends SimSolrCloudTestCase {
     assertEquals(response.get("result").toString(), "success");
 
     CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection("testNodeAdded",
-        "conf",1, 2);
+        "conf",1, 4);
     create.process(solrClient);
 
     CloudTestUtils.waitForState(cluster, "Timed out waiting for replicas of new collection to be active",
@@ -292,7 +292,7 @@ public class TestComputePlanAction extends SimSolrCloudTestCase {
     setClusterPolicyCommand = "{" +
         " 'set-cluster-policy': [" +
         "      {'cores':'<10', 'node':'#ANY'}," +
-        "      {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}," +
+        "      {'replica':'<3', 'shard': '#EACH', 'node': '#ANY'}," +
         "      {'nodeRole':'overseer', 'replica':0}" +
         "    ]" +
         "}";

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6bbce38b/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
index 0fe5ad7..392bdfc 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
@@ -48,7 +48,7 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 {
 
   @Test
   public void test() throws Exception {
-    MetricsHandler handler = new MetricsHandler(h.getCoreContainer());
+    MetricsHandler handler = new MetricsHandler(h.getCoreContainer().getMetricManager());
 
     SolrQueryResponse resp = new SolrQueryResponse();
     handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", MetricsHandler.COMPACT_PARAM, "false", CommonParams.WT, "json"), resp);
@@ -179,7 +179,7 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 {
 
   @Test
   public void testCompact() throws Exception {
-    MetricsHandler handler = new MetricsHandler(h.getCoreContainer());
+    MetricsHandler handler = new MetricsHandler(h.getCoreContainer().getMetricManager());
 
     SolrQueryResponse resp = new SolrQueryResponse();
     handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", MetricsHandler.COMPACT_PARAM, "true"), resp);
@@ -197,7 +197,7 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 {
   public void testPropertyFilter() throws Exception {
     assertQ(req("*:*"), "//result[@numFound='0']");
 
-    MetricsHandler handler = new MetricsHandler(h.getCoreContainer());
+    MetricsHandler handler = new MetricsHandler(h.getCoreContainer().getMetricManager());
 
     SolrQueryResponse resp = new SolrQueryResponse();
     handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json",
@@ -234,7 +234,7 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 {
 
   @Test
   public void testKeyMetrics() throws Exception {
-    MetricsHandler handler = new MetricsHandler(h.getCoreContainer());
+    MetricsHandler handler = new MetricsHandler(h.getCoreContainer().getMetricManager());
 
     String key1 = "solr.core.collection1:CACHE.core.fieldCache";
     SolrQueryResponse resp = new SolrQueryResponse();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6bbce38b/solr/core/src/test/org/apache/solr/handler/admin/MetricsHistoryHandlerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/MetricsHistoryHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/MetricsHistoryHandlerTest.java
new file mode 100644
index 0000000..e1e230f
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/admin/MetricsHistoryHandlerTest.java
@@ -0,0 +1,126 @@
+/*
+ * 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.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.cloud.SolrCloudManager;
+import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.cloud.CloudTestUtils;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.cloud.autoscaling.sim.SimCloudManager;
+import org.apache.solr.common.params.CollectionAdminParams;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.core.SolrInfoBean;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.util.LogLevel;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.rrd4j.core.RrdDb;
+
+/**
+ *
+ */
+@LogLevel("org.apache.solr.cloud=DEBUG")
+public class MetricsHistoryHandlerTest extends SolrCloudTestCase {
+
+  private static SolrCloudManager cloudManager;
+  private static SolrMetricManager metricManager;
+  private static TimeSource timeSource;
+  private static SolrClient solrClient;
+  private static boolean simulated;
+  private static int SPEED;
+
+  private static MetricsHistoryHandler handler;
+  private static MetricsHandler metricsHandler;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    simulated = random().nextBoolean();
+    Map<String, Object> args = new HashMap<>();
+    args.put(MetricsHistoryHandler.SYNC_PERIOD_PROP, 1);
+    args.put(MetricsHistoryHandler.COLLECT_PERIOD_PROP, 1);
+    if (simulated) {
+      SPEED = 50;
+      cloudManager = SimCloudManager.createCluster(1, TimeSource.get("simTime:" + SPEED));
+      // wait for defaults to be applied - due to accelerated time sometimes we may miss this
+      cloudManager.getTimeSource().sleep(10000);
+      AutoScalingConfig cfg = cloudManager.getDistribStateManager().getAutoScalingConfig();
+      assertFalse("autoscaling config is empty", cfg.isEmpty());
+      metricManager = ((SimCloudManager)cloudManager).getMetricManager();
+      solrClient = ((SimCloudManager)cloudManager).simGetSolrClient();
+      // need to register the factory here, before we start the real cluster
+      metricsHandler = new MetricsHandler(metricManager);
+      handler = new MetricsHistoryHandler(cloudManager.getClusterStateProvider().getLiveNodes().iterator().next(),
+          metricsHandler, solrClient, cloudManager, args);
+      handler.initializeMetrics(metricManager, SolrInfoBean.Group.node.toString(), "", CommonParams.METRICS_HISTORY_PATH);
+    }
+    configureCluster(1)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
+    if (!simulated) {
+      cloudManager = cluster.getJettySolrRunner(0).getCoreContainer().getZkController().getSolrCloudManager();
+      metricManager = cluster.getJettySolrRunner(0).getCoreContainer().getMetricManager();
+      solrClient = cluster.getSolrClient();
+      metricsHandler = new MetricsHandler(metricManager);
+      handler = new MetricsHistoryHandler(cluster.getJettySolrRunner(0).getNodeName(), metricsHandler, solrClient, cloudManager, args);
+      handler.initializeMetrics(metricManager, SolrInfoBean.Group.node.toString(), "", CommonParams.METRICS_HISTORY_PATH);
+      SPEED = 1;
+    }
+    timeSource = cloudManager.getTimeSource();
+
+    // create .system collection
+    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(CollectionAdminParams.SYSTEM_COLL,
+        "conf", 1, 1);
+    create.process(solrClient);
+    CloudTestUtils.waitForState(cloudManager, "failed to create " + CollectionAdminParams.SYSTEM_COLL,
+        CollectionAdminParams.SYSTEM_COLL, CloudTestUtils.clusterShape(1, 1));
+  }
+
+  @AfterClass
+  public static void teardown() throws Exception {
+    if (handler != null) {
+      handler.close();
+    }
+    if (simulated) {
+      cloudManager.close();
+    }
+  }
+
+  @Test
+  public void testBasic() throws Exception {
+    timeSource.sleep(10000);
+    List<String> list = handler.getFactory().list(100);
+    // solr.jvm, solr.node, solr.collection..system
+    assertEquals(list.toString(), 3, list.size());
+    for (String path : list) {
+      RrdDb db = new RrdDb(MetricsHistoryHandler.URI_PREFIX + path, true, handler.getFactory());
+      int dsCount = db.getDsCount();
+      int arcCount = db.getArcCount();
+      assertTrue("dsCount should be > 0, was " + dsCount, dsCount > 0);
+      assertEquals("arcCount", 5, arcCount);
+      db.close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6bbce38b/solr/core/src/test/org/apache/solr/metrics/rrd/SolrRrdBackendFactoryTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/rrd/SolrRrdBackendFactoryTest.java b/solr/core/src/test/org/apache/solr/metrics/rrd/SolrRrdBackendFactoryTest.java
new file mode 100644
index 0000000..2f5fa13
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/metrics/rrd/SolrRrdBackendFactoryTest.java
@@ -0,0 +1,182 @@
+/*
+ * 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.metrics.rrd;
+
+import java.util.Date;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.params.CollectionAdminParams;
+import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.util.MockSearchableSolrClient;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.rrd4j.ConsolFun;
+import org.rrd4j.DsType;
+import org.rrd4j.core.FetchData;
+import org.rrd4j.core.FetchRequest;
+import org.rrd4j.core.RrdDb;
+import org.rrd4j.core.RrdDef;
+import org.rrd4j.core.Sample;
+
+/**
+ *
+ */
+public class SolrRrdBackendFactoryTest extends SolrTestCaseJ4 {
+
+  private SolrRrdBackendFactory factory;
+  private MockSearchableSolrClient solrClient;
+  private TimeSource timeSource;
+
+  @Before
+  public void setup() {
+    solrClient = new MockSearchableSolrClient();
+    if (random().nextBoolean()) {
+      timeSource = TimeSource.NANO_TIME;
+    } else {
+      timeSource = TimeSource.get("simTime:50");
+    }
+    factory = new SolrRrdBackendFactory(solrClient, CollectionAdminParams.SYSTEM_COLL, 1, timeSource);
+  }
+
+  @After
+  public void teardown() throws Exception {
+    if (factory != null) {
+      factory.close();
+    }
+  }
+
+  private RrdDef createDef() {
+    RrdDef def = new RrdDef("solr:foo", 60);
+    def.addDatasource("one", DsType.COUNTER, 120, Double.NaN, Double.NaN);
+    def.addDatasource("two", DsType.GAUGE, 120, Double.NaN, Double.NaN);
+    def.addArchive(ConsolFun.AVERAGE, 0.5, 1, 120); // 2 hours
+    def.addArchive(ConsolFun.AVERAGE, 0.5, 10, 288); // 48 hours
+    def.addArchive(ConsolFun.AVERAGE, 0.5, 60, 336); // 2 weeks
+    def.addArchive(ConsolFun.AVERAGE, 0.5, 240, 180); // 2 months
+    return def;
+  }
+
+  @Test
+  public void testBasic() throws Exception {
+    RrdDb db = new RrdDb(createDef(), factory);
+    List<String> list = factory.list(100);
+    assertEquals(list.toString(), 1, list.size());
+    assertEquals(list.toString(), "foo", list.get(0));
+    timeSource.sleep(2000);
+    // there should be one sync data
+    assertEquals(solrClient.docs.toString(), 1, solrClient.docs.size());
+    String id = SolrRrdBackendFactory.ID_PREFIX + SolrRrdBackendFactory.ID_SEP + "foo";
+    SolrInputDocument doc = solrClient.docs.get(CollectionAdminParams.SYSTEM_COLL).get(id);
+    long timestamp = ((Date)doc.getFieldValue("timestamp")).getTime();
+    timeSource.sleep(2000);
+    SolrInputDocument newDoc = solrClient.docs.get(CollectionAdminParams.SYSTEM_COLL).get(id);
+    assertEquals(newDoc.toString(), newDoc, doc);
+    long firstTimestamp = TimeUnit.SECONDS.convert(timestamp, TimeUnit.MILLISECONDS);
+    long lastTimestamp = firstTimestamp + 60;
+    // update the db
+    Sample s = db.createSample();
+    for (int i = 0; i < 100; i++) {
+      s.setTime(lastTimestamp);
+      s.setValue("one", 1000 + i * 60);
+      s.setValue("two", 100);
+      s.update();
+      lastTimestamp = lastTimestamp + 60;
+    }
+    timeSource.sleep(3000);
+    newDoc = solrClient.docs.get(CollectionAdminParams.SYSTEM_COLL).get(id);
+    assertFalse(newDoc.toString(), newDoc.equals(doc));
+    long newTimestamp = ((Date)newDoc.getFieldValue("timestamp")).getTime();
+    assertNotSame(newTimestamp, timestamp);
+    FetchRequest fr = db.createFetchRequest(ConsolFun.AVERAGE, firstTimestamp + 60, lastTimestamp - 60, 60);
+    FetchData fd = fr.fetchData();
+    int rowCount = fd.getRowCount();
+    double[] one = fd.getValues("one");
+    assertEquals("one", 101, one.length);
+    assertEquals(Double.NaN, one[0], 0.00001);
+    assertEquals(Double.NaN, one[100], 0.00001);
+    for (int i = 1; i < 100; i++) {
+      assertEquals(1.0, one[i], 0.00001);
+    }
+    double[] two = fd.getValues("two");
+    assertEquals(Double.NaN, two[100], 0.00001);
+    for (int i = 1; i < 100; i++) {
+      assertEquals("wrong value at pos " + i, 100.0, two[i], 0.00001);
+    }
+    db.close();
+
+    // should still be listed
+    list = factory.list(100);
+    assertEquals(list.toString(), 1, list.size());
+    assertEquals(list.toString(), "foo", list.get(0));
+
+    // re-open read-write
+    db = new RrdDb("solr:foo", factory);
+    s = db.createSample();
+    s.setTime(lastTimestamp);
+    s.setValue("one", 7000);
+    s.setValue("two", 100);
+    s.update();
+    timeSource.sleep(3000);
+    // should update
+    timestamp = newTimestamp;
+    doc = newDoc;
+    newDoc = solrClient.docs.get(CollectionAdminParams.SYSTEM_COLL).get(id);
+    assertFalse(newDoc.toString(), newDoc.equals(doc));
+    newTimestamp = ((Date)newDoc.getFieldValue("timestamp")).getTime();
+    assertNotSame(newTimestamp, timestamp);
+    fr = db.createFetchRequest(ConsolFun.AVERAGE, firstTimestamp + 60, lastTimestamp, 60);
+    fd = fr.fetchData();
+    rowCount = fd.getRowCount();
+    one = fd.getValues("one");
+    assertEquals("one", 102, one.length);
+    assertEquals(Double.NaN, one[0], 0.00001);
+    assertEquals(Double.NaN, one[101], 0.00001);
+    for (int i = 1; i < 101; i++) {
+      assertEquals(1.0, one[i], 0.00001);
+    }
+    two = fd.getValues("two");
+    assertEquals(Double.NaN, two[101], 0.00001);
+    for (int i = 1; i < 101; i++) {
+      assertEquals(100.0, two[i], 0.00001);
+    }
+
+    db.close();
+
+    // open a read-only version of the db
+    RrdDb readOnly = new RrdDb("solr:foo", true, factory);
+    s = readOnly.createSample();
+    s.setTime(lastTimestamp + 120);
+    s.setValue("one", 10000001);
+    s.setValue("two", 100);
+    s.update();
+    // these updates should not be synced
+    timeSource.sleep(3000);
+    doc = newDoc;
+    timestamp = newTimestamp;
+    newDoc = solrClient.docs.get(CollectionAdminParams.SYSTEM_COLL).get(id);
+    assertTrue(newDoc.toString(), newDoc.equals(doc));
+    newTimestamp = ((Date)newDoc.getFieldValue("timestamp")).getTime();
+    assertEquals(newTimestamp, timestamp);
+    readOnly.close();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6bbce38b/solr/core/src/test/org/apache/solr/util/MockSearchableSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/util/MockSearchableSolrClient.java b/solr/core/src/test/org/apache/solr/util/MockSearchableSolrClient.java
new file mode 100644
index 0000000..6fce498
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/util/MockSearchableSolrClient.java
@@ -0,0 +1,117 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.junit.Assert;
+
+/**
+ * Simple mock client that collects added documents and supports simple search by id
+ * (both <code>{!term f=id}</code> and <code>id:</code> syntax is supported) or *:*.
+ */
+public class MockSearchableSolrClient extends SolrClient {
+  public Map<String, Map<String, SolrInputDocument>> docs = new ConcurrentHashMap<>();
+
+  public void clear() {
+    docs.clear();
+  }
+
+  @Override
+  public synchronized NamedList<Object> request(SolrRequest request, String coll) throws SolrServerException, IOException {
+    if (coll == null) {
+      if (request.getParams() != null) {
+        coll = request.getParams().get("collection");
+      }
+    }
+    if (coll == null) {
+      coll = "";
+    }
+    final String collection = coll;
+    NamedList<Object> res = new NamedList<>();
+    if (request instanceof UpdateRequest) {
+      List<SolrInputDocument> docList = ((UpdateRequest) request).getDocuments();
+      if (docList != null) {
+        docList.forEach(doc -> {
+          String id = (String) doc.getFieldValue("id");
+          Assert.assertNotNull(doc.toString(), id);
+          docs.computeIfAbsent(collection, c -> new LinkedHashMap<>()).put(id, doc);
+        });
+      }
+    } else if (request instanceof QueryRequest) {
+      SolrParams params = request.getParams();
+      if (params == null) {
+        throw new UnsupportedOperationException("invalid request, no params: " + request);
+      }
+      String query = params.get("q");
+      final SolrDocumentList lst = new SolrDocumentList();
+      if (query != null) {
+        if (query.startsWith("{!term f=id}") || query.startsWith("id:")) {
+          String id;
+          if (query.startsWith("{!")) {
+            id = query.substring(12);
+          } else {
+            id = query.substring(3);
+          }
+          Map<String, SolrInputDocument> collDocs = docs.get(collection);
+          if (collDocs != null) {
+            SolrInputDocument doc = collDocs.get(id);
+            if (doc != null) {
+              SolrDocument d = new SolrDocument();
+              doc.forEach((k, f) -> f.forEach(v -> d.addField(k, v)));
+              lst.add(d);
+              lst.setNumFound(1);
+            }
+          }
+        } else if (query.equals("*:*")) {
+          Map<String, SolrInputDocument> collDocs = docs.get(collection);
+          if (collDocs != null) {
+            lst.setNumFound(collDocs.size());
+            collDocs.values().forEach(doc -> {
+              SolrDocument d = new SolrDocument();
+              doc.forEach((k, f) -> f.forEach(v -> d.addField(k, v)));
+              lst.add(d);
+            });
+          }
+        }
+      }
+      res.add("response", lst);
+    } else {
+      throw new UnsupportedOperationException("Unsupported request type: " + request.getClass() + ":" + request);
+    }
+    return res;
+  }
+
+  @Override
+  public void close() throws IOException {
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6bbce38b/solr/licenses/rrd4j-3.2.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/rrd4j-3.2.jar.sha1 b/solr/licenses/rrd4j-3.2.jar.sha1
new file mode 100644
index 0000000..5304919
--- /dev/null
+++ b/solr/licenses/rrd4j-3.2.jar.sha1
@@ -0,0 +1 @@
+8d480d5aa87b3d358862b78d6fa3660396220dc7

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6bbce38b/solr/licenses/rrd4j-LICENSE-ASL.txt
----------------------------------------------------------------------
diff --git a/solr/licenses/rrd4j-LICENSE-ASL.txt b/solr/licenses/rrd4j-LICENSE-ASL.txt
new file mode 100644
index 0000000..7a4a3ea
--- /dev/null
+++ b/solr/licenses/rrd4j-LICENSE-ASL.txt
@@ -0,0 +1,202 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed 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.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6bbce38b/solr/licenses/rrd4j-NOTICE.txt
----------------------------------------------------------------------
diff --git a/solr/licenses/rrd4j-NOTICE.txt b/solr/licenses/rrd4j-NOTICE.txt
new file mode 100644
index 0000000..841dd30
--- /dev/null
+++ b/solr/licenses/rrd4j-NOTICE.txt
@@ -0,0 +1,2 @@
+GitHub: https://github.com/rrd4j/rrd4j/
+Maven-generated site: http://rrd4j.org/site/

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6bbce38b/solr/solr-ref-guide/src/images/metrics-history/loadAvg-60s.png
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/images/metrics-history/loadAvg-60s.png b/solr/solr-ref-guide/src/images/metrics-history/loadAvg-60s.png
new file mode 100644
index 0000000..4812ede
Binary files /dev/null and b/solr/solr-ref-guide/src/images/metrics-history/loadAvg-60s.png differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6bbce38b/solr/solr-ref-guide/src/images/metrics-history/memHeap-60s.png
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/images/metrics-history/memHeap-60s.png b/solr/solr-ref-guide/src/images/metrics-history/memHeap-60s.png
new file mode 100644
index 0000000..cbf1a2e
Binary files /dev/null and b/solr/solr-ref-guide/src/images/metrics-history/memHeap-60s.png differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6bbce38b/solr/solr-ref-guide/src/images/metrics-history/query-graph-10min.png
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/images/metrics-history/query-graph-10min.png b/solr/solr-ref-guide/src/images/metrics-history/query-graph-10min.png
new file mode 100644
index 0000000..81aa1b4
Binary files /dev/null and b/solr/solr-ref-guide/src/images/metrics-history/query-graph-10min.png differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6bbce38b/solr/solr-ref-guide/src/images/metrics-history/query-graph-60s.png
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/images/metrics-history/query-graph-60s.png b/solr/solr-ref-guide/src/images/metrics-history/query-graph-60s.png
new file mode 100644
index 0000000..31d230e
Binary files /dev/null and b/solr/solr-ref-guide/src/images/metrics-history/query-graph-60s.png differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6bbce38b/solr/solr-ref-guide/src/images/metrics-history/update-graph-60s.png
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/images/metrics-history/update-graph-60s.png b/solr/solr-ref-guide/src/images/metrics-history/update-graph-60s.png
new file mode 100644
index 0000000..594f02d
Binary files /dev/null and b/solr/solr-ref-guide/src/images/metrics-history/update-graph-60s.png differ