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

[01/44] lucene-solr:jira/solr-9854: fix RangeFieldQuery.scorer to return null if no docs in a segment indexed the field

Repository: lucene-solr
Updated Branches:
  refs/heads/jira/solr-9854 ce8ab8c63 -> fa87f6a00


fix RangeFieldQuery.scorer to return null if no docs in a segment indexed the field


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

Branch: refs/heads/jira/solr-9854
Commit: 9aa5b734c38ed0b9327577bd2b1413d448230eab
Parents: 8c79ab2
Author: Nicholas Knize <nk...@gmail.com>
Authored: Tue Dec 13 15:07:06 2016 -0600
Committer: Nicholas Knize <nk...@gmail.com>
Committed: Tue Dec 13 15:07:06 2016 -0600

----------------------------------------------------------------------
 .../src/java/org/apache/lucene/document/RangeFieldQuery.java       | 1 +
 .../test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java | 2 +-
 2 files changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9aa5b734/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java b/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java
index 7ebdec4..5249191 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java
@@ -165,6 +165,7 @@ abstract class RangeFieldQuery extends Query {
         FieldInfo fieldInfo = reader.getFieldInfos().fieldInfo(field);
         if (fieldInfo == null) {
           // no docs in this segment indexed this field
+          return null;
         }
         checkFieldInfo(fieldInfo);
         boolean allDocsMatch = true;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9aa5b734/lucene/sandbox/src/test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java b/lucene/sandbox/src/test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java
index ff61ff6..ceafd53 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java
@@ -93,7 +93,7 @@ public abstract class BaseRangeFieldQueryTestCase extends LuceneTestCase {
         ranges[id] = new Range[] {nextRange(dimensions)};
       }
       if (x == 17) {
-        // dome docs don't have a box:
+        // some docs don't have a box:
         ranges[id][0].isMissing = true;
         if (VERBOSE) {
           System.out.println("  id=" + id + " is missing");


[41/44] lucene-solr:jira/solr-9854: Squashed commit of branch 'feature/metrics', containing: SOLR-4735: Improve Solr metrics reporting SOLR-9812: Implement /admin/metrics API SOLR-9805: Use metrics-jvm library to instrument jvm internals

Posted by ab...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/java/org/apache/solr/metrics/SolrMetricReporter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrMetricReporter.java b/solr/core/src/java/org/apache/solr/metrics/SolrMetricReporter.java
new file mode 100644
index 0000000..ff2d3fc
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrMetricReporter.java
@@ -0,0 +1,83 @@
+/*
+ * 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;
+
+import java.io.Closeable;
+
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.util.SolrPluginUtils;
+import org.apache.solr.util.plugin.PluginInfoInitialized;
+
+/**
+ * Interface for 'pluggable' metric reporters.
+ */
+public abstract class SolrMetricReporter implements Closeable, PluginInfoInitialized {
+
+  protected final String registryName;
+  protected final SolrMetricManager metricManager;
+  protected PluginInfo pluginInfo;
+
+  /**
+   * Create a reporter for metrics managed in a named registry.
+   * @param registryName registry to use, one of registries managed by
+   *                     {@link SolrMetricManager}
+   */
+  protected SolrMetricReporter(SolrMetricManager metricManager, String registryName) {
+    this.registryName = registryName;
+    this.metricManager = metricManager;
+  }
+
+  /**
+   * Initializes a {@link SolrMetricReporter} with the plugin's configuration.
+   *
+   * @param pluginInfo the plugin's configuration
+   */
+  @SuppressWarnings("unchecked")
+  public void init(PluginInfo pluginInfo) {
+    if (pluginInfo != null) {
+      this.pluginInfo = pluginInfo.copy();
+      if (this.pluginInfo.initArgs != null) {
+        SolrPluginUtils.invokeSetters(this, this.pluginInfo.initArgs);
+      }
+    }
+    validate();
+  }
+
+  /**
+   * Get the effective {@link PluginInfo} instance that was used for
+   * initialization of this plugin.
+   * @return plugin info, or null if not yet initialized.
+   */
+  public PluginInfo getPluginInfo() {
+    return pluginInfo;
+  }
+
+  /**
+   * Validates that the reporter has been correctly configured.
+   *
+   * @throws IllegalStateException if the reporter is not properly configured
+   */
+  protected abstract void validate() throws IllegalStateException;
+
+  @Override
+  public String toString() {
+    return getClass().getName() + "{" +
+        "registryName='" + registryName + '\'' +
+        ", pluginInfo=" + pluginInfo +
+        '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/java/org/apache/solr/metrics/package-info.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/package-info.java b/solr/core/src/java/org/apache/solr/metrics/package-info.java
new file mode 100644
index 0000000..a007b8b
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/metrics/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+/**
+ * The {@link org.apache.solr.metrics.SolrCoreMetricManager} is responsible for
+ * collecting metrics from {@link org.apache.solr.metrics.SolrMetricProducer}'s
+ * and exposing metrics to {@link org.apache.solr.metrics.SolrMetricReporter}'s.
+ */
+package org.apache.solr.metrics;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/java/org/apache/solr/metrics/reporters/SolrJmxReporter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/reporters/SolrJmxReporter.java b/solr/core/src/java/org/apache/solr/metrics/reporters/SolrJmxReporter.java
new file mode 100644
index 0000000..47fbf11
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/metrics/reporters/SolrJmxReporter.java
@@ -0,0 +1,284 @@
+/*
+ * 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.reporters;
+
+import javax.management.MBeanServer;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.lang.management.ManagementFactory;
+import java.util.Locale;
+
+import com.codahale.metrics.JmxReporter;
+import com.codahale.metrics.ObjectNameFactory;
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.metrics.SolrMetricInfo;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.metrics.SolrMetricReporter;
+import org.apache.solr.util.JmxUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A {@link SolrMetricReporter} that finds (or creates) a MBeanServer from
+ * the given configuration and registers metrics to it with JMX.
+ */
+public class SolrJmxReporter extends SolrMetricReporter {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private String domain;
+  private String agentId;
+  private String serviceUrl;
+
+  private JmxReporter reporter;
+  private MBeanServer mBeanServer;
+
+  /**
+   * Creates a new instance of {@link SolrJmxReporter}.
+   *
+   * @param registryName name of the registry to report
+   */
+  public SolrJmxReporter(SolrMetricManager metricManager, String registryName) {
+    super(metricManager, registryName);
+    setDomain(registryName);
+  }
+
+  /**
+   * Initializes the reporter by finding (or creating) a MBeanServer
+   * and registering the metricManager's metric registry.
+   *
+   * @param pluginInfo the configuration for the reporter
+   */
+  @Override
+  public synchronized void init(PluginInfo pluginInfo) {
+    super.init(pluginInfo);
+
+    if (serviceUrl != null && agentId != null) {
+      ManagementFactory.getPlatformMBeanServer(); // Ensure at least one MBeanServer is available.
+      mBeanServer = JmxUtil.findFirstMBeanServer();
+      log.warn("No more than one of serviceUrl(%s) and agentId(%s) should be configured, using first MBeanServer instead of configuration.",
+          serviceUrl, agentId, mBeanServer);
+    }
+    else if (serviceUrl != null) {
+      try {
+        mBeanServer = JmxUtil.findMBeanServerForServiceUrl(serviceUrl);
+      } catch (IOException e) {
+        log.warn("findMBeanServerForServiceUrl(%s) exception: %s", serviceUrl, e);
+        mBeanServer = null;
+      }
+    }
+    else if (agentId != null) {
+      mBeanServer = JmxUtil.findMBeanServerForAgentId(agentId);
+    } else {
+      ManagementFactory.getPlatformMBeanServer(); // Ensure at least one MBeanServer is available.
+      mBeanServer = JmxUtil.findFirstMBeanServer();
+      log.warn("No serviceUrl or agentId was configured, using first MBeanServer.", mBeanServer);
+    }
+
+    if (mBeanServer == null) {
+      log.warn("No JMX server found. Not exposing Solr metrics.");
+      return;
+    }
+
+    JmxObjectNameFactory jmxObjectNameFactory = new JmxObjectNameFactory(pluginInfo.name, domain);
+
+    reporter = JmxReporter.forRegistry(metricManager.registry(registryName))
+                          .registerWith(mBeanServer)
+                          .inDomain(domain)
+                          .createsObjectNamesWith(jmxObjectNameFactory)
+                          .build();
+    reporter.start();
+
+    log.info("JMX monitoring enabled at server: " + mBeanServer);
+  }
+
+  /**
+   * Stops the reporter from publishing metrics.
+   */
+  @Override
+  public synchronized void close() {
+    if (reporter != null) {
+      reporter.close();
+      reporter = null;
+    }
+  }
+
+  /**
+   * Validates that the reporter has been correctly configured.
+   * Note that all configurable arguments are currently optional.
+   *
+   * @throws IllegalStateException if the reporter is not properly configured
+   */
+  @Override
+  protected void validate() throws IllegalStateException {
+    // Nothing to validate
+  }
+
+  /**
+   * Sets the domain with which MBeans are published. If none is set,
+   * the domain defaults to the name of the core.
+   *
+   * @param domain the domain
+   */
+  public void setDomain(String domain) {
+    if (domain != null) {
+      this.domain = domain;
+    } else {
+      this.domain = registryName;
+    }
+  }
+
+  /**
+   * Sets the service url for a JMX server.
+   * Note that this configuration is optional.
+   *
+   * @param serviceUrl the service url
+   */
+  public void setServiceUrl(String serviceUrl) {
+    this.serviceUrl = serviceUrl;
+  }
+
+  /**
+   * Sets the agent id for a JMX server.
+   * Note that this configuration is optional.
+   *
+   * @param agentId the agent id
+   */
+  public void setAgentId(String agentId) {
+    this.agentId = agentId;
+  }
+
+  /**
+   * Retrieves the reporter's MBeanServer.
+   *
+   * @return the reporter's MBeanServer
+   */
+  public MBeanServer getMBeanServer() {
+    return mBeanServer;
+  }
+
+  @Override
+  public String toString() {
+    return String.format(Locale.ENGLISH, "[%s@%s: domain = %s, service url = %s, agent id = %s]",
+        getClass().getName(), Integer.toHexString(hashCode()), domain, serviceUrl, agentId);
+  }
+
+  /**
+   * Factory to create MBean names for a given metric.
+   */
+  private static class JmxObjectNameFactory implements ObjectNameFactory {
+
+    private final String domain;
+    private final String[] subdomains;
+    private final String reporterName;
+
+    JmxObjectNameFactory(String reporterName, String domain) {
+      this.reporterName = reporterName;
+      this.domain = domain;
+      this.subdomains = domain.split("\\.");
+    }
+
+    /**
+     * Create a hierarchical name of a metric.
+     *
+     * @param type    metric class, eg. "counters"
+     * @param currentDomain  JMX domain
+     * @param name    metric name
+     */
+    @Override
+    public ObjectName createName(String type, String currentDomain, String name) {
+      SolrMetricInfo metricInfo = SolrMetricInfo.of(name);
+
+      // It turns out that ObjectName(String) mostly preserves key ordering
+      // as specified in the constructor (except for the 'type' key that ends
+      // up at top level) - unlike ObjectName(String, Map) constructor
+      // that seems to have a mind of its own...
+      StringBuilder sb = new StringBuilder();
+      if (domain.equals(currentDomain)) {
+        if (subdomains != null && subdomains.length > 1) {
+          // use only first segment as domain
+          sb.append(subdomains[0]);
+          sb.append(':');
+          // use remaining segments as properties
+          for (int i = 1; i < subdomains.length; i++) {
+            if (i > 1) {
+              sb.append(',');
+            }
+            sb.append("dom");
+            sb.append(String.valueOf(i));
+            sb.append('=');
+            sb.append(subdomains[i]);
+          }
+          sb.append(','); // separate from other properties
+        } else {
+          sb.append(currentDomain);
+          sb.append(':');
+        }
+      } else {
+        sb.append(currentDomain);
+        sb.append(':');
+      }
+      sb.append("reporter=");
+      sb.append(reporterName);
+      sb.append(',');
+      if (metricInfo != null) {
+        sb.append("category=");
+        sb.append(metricInfo.category.toString());
+        sb.append(",scope=");
+        sb.append(metricInfo.scope);
+        // we could also split by type, but don't call it 'type' :)
+        // sb.append(",class=");
+        //sb.append(type);
+        sb.append(",name=");
+        sb.append(metricInfo.name);
+      } else {
+        // make dotted names into hierarchies
+        String[] path = name.split("\\.");
+        for (int i = 0; i < path.length - 1; i++) {
+          if (i > 0) {
+            sb.append(',');
+          }
+          sb.append("name"); sb.append(String.valueOf(i));
+          sb.append('=');
+          sb.append(path[i]);
+        }
+        if (path.length > 1) {
+          sb.append(',');
+        }
+        // split by type
+        // sb.append("class=");
+        // sb.append(type);
+        sb.append("name=");
+        sb.append(path[path.length - 1]);
+      }
+
+      ObjectName objectName;
+
+      try {
+        objectName = new ObjectName(sb.toString());
+      } catch (MalformedObjectNameException e) {
+        throw new RuntimeException(sb.toString(), e);
+      }
+
+      return objectName;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/java/org/apache/solr/metrics/reporters/package-info.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/reporters/package-info.java b/solr/core/src/java/org/apache/solr/metrics/reporters/package-info.java
new file mode 100644
index 0000000..5ad7a80
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/metrics/reporters/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * This package houses 'pluggable' metric reporters that
+ * inherit from the {@link org.apache.solr.metrics.SolrMetricReporter} class.
+ */
+package org.apache.solr.metrics.reporters;

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
index e8c4657..dbc4b35 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
@@ -16,6 +16,7 @@
  */
 package org.apache.solr.servlet;
 
+import javax.management.MBeanServer;
 import javax.servlet.FilterChain;
 import javax.servlet.FilterConfig;
 import javax.servlet.ServletException;
@@ -33,6 +34,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.lang.invoke.MethodHandles;
+import java.lang.management.ManagementFactory;
 import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.time.Instant;
@@ -45,6 +47,12 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import com.codahale.metrics.jvm.BufferPoolMetricSet;
+import com.codahale.metrics.jvm.ClassLoadingGaugeSet;
+import com.codahale.metrics.jvm.FileDescriptorRatioGauge;
+import com.codahale.metrics.jvm.GarbageCollectorMetricSet;
+import com.codahale.metrics.jvm.MemoryUsageGaugeSet;
+import com.codahale.metrics.jvm.ThreadStatesGaugeSet;
 import org.apache.commons.io.FileCleaningTracker;
 import org.apache.commons.io.input.CloseShieldInputStream;
 import org.apache.commons.io.output.CloseShieldOutputStream;
@@ -58,8 +66,10 @@ import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.NodeConfig;
 import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.SolrInfoMBean;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.core.SolrXmlConfig;
+import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.request.SolrRequestInfo;
 import org.apache.solr.security.AuthenticationPlugin;
 import org.apache.solr.security.PKIAuthenticationPlugin;
@@ -157,6 +167,7 @@ public class SolrDispatchFilter extends BaseSolrFilter {
       this.cores = createCoreContainer(solrHome == null ? SolrResourceLoader.locateSolrHome() : Paths.get(solrHome),
                                        extraProperties);
       this.httpClient = cores.getUpdateShardHandler().getHttpClient();
+      setupJvmMetrics();
       log.debug("user.dir=" + System.getProperty("user.dir"));
     }
     catch( Throwable t ) {
@@ -171,6 +182,22 @@ public class SolrDispatchFilter extends BaseSolrFilter {
     log.trace("SolrDispatchFilter.init() done");
   }
 
+  private void setupJvmMetrics()  {
+    MBeanServer platformMBeanServer = ManagementFactory.getPlatformMBeanServer();
+    SolrMetricManager metricManager = cores.getMetricManager();
+    try {
+      String registry = SolrMetricManager.getRegistryName(SolrInfoMBean.Group.jvm);
+      metricManager.registerAll(registry, new BufferPoolMetricSet(platformMBeanServer), true, "bufferPools");
+      metricManager.registerAll(registry, new ClassLoadingGaugeSet(), true, "classLoading");
+      metricManager.register(registry, new FileDescriptorRatioGauge(), true, "fileDescriptorRatio");
+      metricManager.registerAll(registry, new GarbageCollectorMetricSet(), true, "gc");
+      metricManager.registerAll(registry, new MemoryUsageGaugeSet(), true, "memory");
+      metricManager.registerAll(registry, new ThreadStatesGaugeSet(), true, "threads"); // todo should we use CachedThreadStatesGaugeSet instead?
+    } catch (Exception e) {
+      log.warn("Error registering JVM metrics", e);
+    }
+  }
+
   private void logWelcomeBanner() {
     log.info(" ___      _       Welcome to Apache Solr\u2122 version {}", solrVersion());
     log.info("/ __| ___| |_ _   Starting in {} mode on port {}", isCloudMode() ? "cloud" : "standalone", getSolrPort());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/java/org/apache/solr/util/JmxUtil.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/JmxUtil.java b/solr/core/src/java/org/apache/solr/util/JmxUtil.java
new file mode 100644
index 0000000..02a070d
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/util/JmxUtil.java
@@ -0,0 +1,78 @@
+/*
+ * 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 javax.management.MBeanServer;
+import javax.management.MBeanServerFactory;
+import javax.management.remote.JMXConnectorServer;
+import javax.management.remote.JMXConnectorServerFactory;
+import javax.management.remote.JMXServiceURL;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Utility methods to find a MBeanServer.
+ *
+ * This was factored out from {@link org.apache.solr.core.JmxMonitoredMap}
+ * and can eventually replace the logic used there.
+ */
+public final class JmxUtil {
+
+  /**
+   * Retrieve the first MBeanServer found.
+   *
+   * @return the first MBeanServer found
+   */
+  public static MBeanServer findFirstMBeanServer() {
+    return findMBeanServerForAgentId(null);
+  }
+
+  /**
+   * Find a MBeanServer given a service url.
+   *
+   * @param serviceUrl the service url
+   * @return a MBeanServer
+   */
+  public static MBeanServer findMBeanServerForServiceUrl(String serviceUrl) throws IOException {
+    if (serviceUrl == null) {
+      return null;
+    }
+
+    MBeanServer server = MBeanServerFactory.newMBeanServer();
+    JMXConnectorServer connector = JMXConnectorServerFactory
+        .newJMXConnectorServer(new JMXServiceURL(serviceUrl), null, server);
+    connector.start();
+
+    return server;
+  }
+
+  /**
+   * Find a MBeanServer given an agent id.
+   *
+   * @param agentId the agent id
+   * @return a MBeanServer
+   */
+  public static MBeanServer findMBeanServerForAgentId(String agentId) {
+    List<MBeanServer> servers = MBeanServerFactory.findMBeanServer(agentId);
+    if (servers == null || servers.isEmpty()) {
+      return null;
+    }
+
+    return servers.get(0);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java b/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java
new file mode 100644
index 0000000..62f5776
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.util.stats;
+
+import java.util.List;
+import java.util.Map;
+import java.util.SortedSet;
+import java.util.concurrent.TimeUnit;
+
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.Meter;
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricFilter;
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.Snapshot;
+import com.codahale.metrics.Timer;
+import org.apache.solr.common.util.NamedList;
+
+/**
+ * Metrics specific utility functions.
+ */
+public class MetricUtils {
+
+  /**
+   * Adds metrics from a Timer to a NamedList, using well-known names.
+   * @param lst The NamedList to add the metrics data to
+   * @param timer The Timer to extract the metrics from
+   */
+  public static void addMetrics(NamedList<Object> lst, Timer timer) {
+    Snapshot snapshot = timer.getSnapshot();
+    lst.add("avgRequestsPerSecond", timer.getMeanRate());
+    lst.add("5minRateRequestsPerSecond", timer.getFiveMinuteRate());
+    lst.add("15minRateRequestsPerSecond", timer.getFifteenMinuteRate());
+    lst.add("avgTimePerRequest", nsToMs(snapshot.getMean()));
+    lst.add("medianRequestTime", nsToMs(snapshot.getMedian()));
+    lst.add("75thPcRequestTime", nsToMs(snapshot.get75thPercentile()));
+    lst.add("95thPcRequestTime", nsToMs(snapshot.get95thPercentile()));
+    lst.add("99thPcRequestTime", nsToMs(snapshot.get99thPercentile()));
+    lst.add("999thPcRequestTime", nsToMs(snapshot.get999thPercentile()));
+  }
+
+  /**
+   * Converts a double representing nanoseconds to a double representing milliseconds.
+   *
+   * @param ns the amount of time in nanoseconds
+   * @return the amount of time in milliseconds
+   */
+  static double nsToMs(double ns) {
+    return ns / TimeUnit.MILLISECONDS.toNanos(1);
+  }
+
+  /**
+   * Returns a NamedList respresentation of the given metric registry. Only those metrics
+   * are converted to NamedList which match at least one of the given MetricFilter instances.
+   *
+   * @param registry      the {@link MetricRegistry} to be converted to NamedList
+   * @param metricFilters a list of {@link MetricFilter} instances
+   * @return a {@link NamedList}
+   */
+  public static NamedList toNamedList(MetricRegistry registry, List<MetricFilter> metricFilters) {
+    NamedList response = new NamedList();
+    Map<String, Metric> metrics = registry.getMetrics();
+    SortedSet<String> names = registry.getNames();
+    names.stream().filter(s -> metricFilters.stream().anyMatch(metricFilter -> metricFilter.matches(s, metrics.get(s)))).forEach(n -> {
+      Metric metric = metrics.get(n);
+      if (metric instanceof Counter) {
+        Counter counter = (Counter) metric;
+        response.add(n, counterToNamedList(counter));
+      } else if (metric instanceof Gauge) {
+        Gauge gauge = (Gauge) metric;
+        response.add(n, gaugeToNamedList(gauge));
+      } else if (metric instanceof Meter) {
+        Meter meter = (Meter) metric;
+        response.add(n, meterToNamedList(meter));
+      } else if (metric instanceof Timer) {
+        Timer timer = (Timer) metric;
+        response.add(n, timerToNamedList(timer));
+      } else if (metric instanceof Histogram) {
+        Histogram histogram = (Histogram) metric;
+        response.add(n, histogramToNamedList(histogram));
+      }
+    });
+    return response;
+  }
+
+  static NamedList histogramToNamedList(Histogram histogram) {
+    NamedList response = new NamedList();
+    Snapshot snapshot = histogram.getSnapshot();
+    response.add("requests", histogram.getCount());
+    response.add("minTime", nsToMs(snapshot.getMin()));
+    response.add("maxTime", nsToMs(snapshot.getMax()));
+    response.add("avgTimePerRequest", nsToMs(snapshot.getMean()));
+    response.add("medianRequestTime", nsToMs(snapshot.getMedian()));
+    response.add("75thPcRequestTime", nsToMs(snapshot.get75thPercentile()));
+    response.add("95thPcRequestTime", nsToMs(snapshot.get95thPercentile()));
+    response.add("99thPcRequestTime", nsToMs(snapshot.get99thPercentile()));
+    response.add("999thPcRequestTime", nsToMs(snapshot.get999thPercentile()));
+    return response;
+  }
+
+  static NamedList timerToNamedList(Timer timer) {
+    NamedList response = new NamedList();
+    addMetrics(response, timer);
+    return response;
+  }
+
+  static NamedList meterToNamedList(Meter meter) {
+    NamedList response = new NamedList();
+    response.add("requests", meter.getCount());
+    response.add("avgRequestsPerSecond", meter.getMeanRate());
+    response.add("1minRateRequestsPerSecond", meter.getOneMinuteRate());
+    response.add("5minRateRequestsPerSecond", meter.getFiveMinuteRate());
+    response.add("15minRateRequestsPerSecond", meter.getFifteenMinuteRate());
+    return response;
+  }
+
+  static NamedList gaugeToNamedList(Gauge gauge) {
+    NamedList response = new NamedList();
+    response.add("value", gauge.getValue());
+    return response;
+  }
+
+  static NamedList counterToNamedList(Counter counter) {
+    NamedList response = new NamedList();
+    response.add("requests", counter.getCount());
+    return response;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/java/org/apache/solr/util/stats/TimerUtils.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/stats/TimerUtils.java b/solr/core/src/java/org/apache/solr/util/stats/TimerUtils.java
deleted file mode 100644
index 243c1ee..0000000
--- a/solr/core/src/java/org/apache/solr/util/stats/TimerUtils.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.util.stats;
-
-import java.util.concurrent.TimeUnit;
-
-import com.codahale.metrics.Snapshot;
-import com.codahale.metrics.Timer;
-import org.apache.solr.common.util.NamedList;
-
-/**
- * Solr specific {@link Timer} utility functions.
- */
-public class TimerUtils {
-
-  /**
-   * Adds metrics from a Timer to a NamedList, using well-known names.
-   * @param lst The NamedList to add the metrics data to
-   * @param timer The Timer to extract the metrics from
-   */
-  public static void addMetrics(NamedList<Object> lst, Timer timer) {
-    Snapshot snapshot = timer.getSnapshot();
-    lst.add("avgRequestsPerSecond", timer.getMeanRate());
-    lst.add("5minRateRequestsPerSecond", timer.getFiveMinuteRate());
-    lst.add("15minRateRequestsPerSecond", timer.getFifteenMinuteRate());
-    lst.add("avgTimePerRequest", nsToMs(snapshot.getMean()));
-    lst.add("medianRequestTime", nsToMs(snapshot.getMedian()));
-    lst.add("75thPcRequestTime", nsToMs(snapshot.get75thPercentile()));
-    lst.add("95thPcRequestTime", nsToMs(snapshot.get95thPercentile()));
-    lst.add("99thPcRequestTime", nsToMs(snapshot.get99thPercentile()));
-    lst.add("999thPcRequestTime", nsToMs(snapshot.get999thPercentile()));
-  }
-
-  /**
-   * Converts a double representing nanoseconds to a double representing milliseconds.
-   *
-   * @param ns the amount of time in nanoseconds
-   * @return the amount of time in milliseconds
-   */
-  static double nsToMs(double ns) {
-    return ns / TimeUnit.MILLISECONDS.toNanos(1);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/test-files/solr/solr-metricreporter.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/solr-metricreporter.xml b/solr/core/src/test-files/solr/solr-metricreporter.xml
new file mode 100644
index 0000000..cd8d737
--- /dev/null
+++ b/solr/core/src/test-files/solr/solr-metricreporter.xml
@@ -0,0 +1,57 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+ 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.
+-->
+
+<solr>
+ <metrics>
+  <!-- this reporter doesn't specify 'group' or 'registry', it will be instantiated for any group. -->
+  <reporter name="universal" class="org.apache.solr.metrics.reporters.MockMetricReporter">
+    <str name="configurable">configured</str>
+  </reporter>
+  <!-- this reporter will be created for both "node" and "core" groups -->
+  <reporter name="multigroup" group="node, core" class="org.apache.solr.metrics.reporters.MockMetricReporter">
+    <str name="configurable">configured</str>
+  </reporter>
+  <!-- this reporter will be created for both "node" and "core.collection1" registries -->
+  <reporter name="multiregistry" registry="node, core.collection1" class="org.apache.solr.metrics.reporters.MockMetricReporter">
+    <str name="configurable">configured</str>
+  </reporter>
+  <reporter name="reporter1" group="jvm" class="org.apache.solr.metrics.reporters.MockMetricReporter">
+    <str name="configurable">configured</str>
+  </reporter>
+  <reporter name="reporter1" group="jetty" class="org.apache.solr.metrics.reporters.MockMetricReporter">
+    <str name="configurable">configured</str>
+  </reporter>
+  <reporter name="reporter1" group="http" class="org.apache.solr.metrics.reporters.MockMetricReporter">
+    <str name="configurable">configured</str>
+  </reporter>
+  <reporter name="reporter1" group="node" class="org.apache.solr.metrics.reporters.MockMetricReporter">
+    <str name="configurable">configured</str>
+  </reporter>
+  <!-- core reporters. -->
+  <reporter name="reporter1" group="core" class="org.apache.solr.metrics.reporters.MockMetricReporter">
+    <str name="configurable">configured</str>
+  </reporter>
+  <reporter name="reporter2" group="core" class="org.apache.solr.metrics.reporters.MockMetricReporter">
+    <str name="configurable">configured</str>
+  </reporter>
+  <!-- reporter for a specific registry -->
+  <reporter name="specific" registry="core.collection1" class="org.apache.solr.metrics.reporters.MockMetricReporter">
+    <str name="configurable">configured</str>
+  </reporter>
+ </metrics>
+</solr>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
new file mode 100644
index 0000000..3667285
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.handler.admin;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.response.SolrQueryResponse;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test for {@link MetricsHandler}
+ */
+public class MetricsHandlerTest extends SolrTestCaseJ4 {
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    initCore("solrconfig.xml", "schema.xml");
+  }
+
+  @Test
+  public void test() throws Exception {
+    MetricsHandler handler = new MetricsHandler(h.getCoreContainer());
+
+    SolrQueryResponse resp = new SolrQueryResponse();
+    handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json"), resp);
+    NamedList values = resp.getValues();
+    assertNotNull(values.get("metrics"));
+    values = (NamedList) values.get("metrics");
+    System.out.println(values);
+    assertNotNull(values.get("solr.jetty"));
+    assertNotNull(values.get("solr.jvm"));
+    assertNotNull(values.get("solr.http"));
+    assertNotNull(values.get("solr.node"));
+    NamedList nl = (NamedList) values.get("solr.core.collection1");
+    assertNotNull(nl);
+    assertNotNull(nl.get("newSearcherErrors")); // counter type
+    assertNotNull(((NamedList) nl.get("newSearcherErrors")).get("requests"));
+    assertEquals(0L, ((NamedList) nl.get("newSearcherErrors")).get("requests"));
+    nl = (NamedList) values.get("solr.node");
+    assertNotNull(nl.get("cores.loaded")); // int gauge
+    assertEquals(1, ((NamedList) nl.get("cores.loaded")).get("value"));
+    assertNotNull(nl.get("QUERYHANDLER./admin/authorization.clientErrors")); // timer type
+    assertEquals(5, ((NamedList) nl.get("QUERYHANDLER./admin/authorization.clientErrors")).size());
+
+    resp = new SolrQueryResponse();
+    handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", "group", "jvm,jetty"), resp);
+    values = resp.getValues();
+    assertNotNull(values.get("metrics"));
+    values = (NamedList) values.get("metrics");
+    assertEquals(2, values.size());
+    assertNotNull(values.get("solr.jetty"));
+    assertNotNull(values.get("solr.jvm"));
+
+    resp = new SolrQueryResponse();
+    handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", "group", "jvm,jetty"), resp);
+    values = resp.getValues();
+    assertNotNull(values.get("metrics"));
+    values = (NamedList) values.get("metrics");
+    assertEquals(2, values.size());
+    assertNotNull(values.get("solr.jetty"));
+    assertNotNull(values.get("solr.jvm"));
+
+    resp = new SolrQueryResponse();
+    handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", "group", "jvm", "group", "jetty"), resp);
+    values = resp.getValues();
+    assertNotNull(values.get("metrics"));
+    values = (NamedList) values.get("metrics");
+    assertEquals(2, values.size());
+    assertNotNull(values.get("solr.jetty"));
+    assertNotNull(values.get("solr.jvm"));
+
+    resp = new SolrQueryResponse();
+    handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", "group", "node", "type", "counter"), resp);
+    values = resp.getValues();
+    assertNotNull(values.get("metrics"));
+    values = (NamedList) values.get("metrics");
+    assertEquals(1, values.size());
+    assertNotNull(values.get("solr.node"));
+    assertNull(values.get("QUERYHANDLER./admin/authorization.errors")); // this is a timer node
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/test/org/apache/solr/metrics/SolrCoreMetricManagerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/SolrCoreMetricManagerTest.java b/solr/core/src/test/org/apache/solr/metrics/SolrCoreMetricManagerTest.java
new file mode 100644
index 0000000..65ffb93
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/metrics/SolrCoreMetricManagerTest.java
@@ -0,0 +1,172 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+import java.util.stream.Collectors;
+
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricRegistry;
+import org.apache.lucene.util.TestUtil;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.params.CoreAdminParams;
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.core.SolrInfoMBean;
+import org.apache.solr.metrics.reporters.MockMetricReporter;
+import org.apache.solr.schema.FieldType;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class SolrCoreMetricManagerTest extends SolrTestCaseJ4 {
+  private static final int MAX_ITERATIONS = 100;
+
+  private SolrCoreMetricManager coreMetricManager;
+  private SolrMetricManager metricManager;
+
+  @Before
+  public void beforeTest() throws Exception {
+    initCore("solrconfig-basic.xml", "schema.xml");
+    coreMetricManager = h.getCore().getCoreMetricManager();
+    metricManager = h.getCore().getCoreDescriptor().getCoreContainer().getMetricManager();
+  }
+
+  @After
+  public void afterTest() throws IOException {
+    coreMetricManager.close();
+    assertTrue(metricManager.getReporters(coreMetricManager.getRegistryName()).isEmpty());
+    deleteCore();
+  }
+
+  @Test
+  public void testRegisterMetrics() {
+    Random random = random();
+
+    String scope = SolrMetricTestUtils.getRandomScope(random);
+    SolrInfoMBean.Category category = SolrMetricTestUtils.getRandomCategory(random);
+    Map<String, Counter> metrics = SolrMetricTestUtils.getRandomMetrics(random);
+    SolrMetricProducer producer = SolrMetricTestUtils.getProducerOf(metricManager, category, scope, metrics);
+    try {
+      coreMetricManager.registerMetricProducer(scope, producer);
+      assertNotNull(scope);
+      assertNotNull(category);
+      assertNotNull(metrics);
+      assertRegistered(scope, metrics, coreMetricManager);
+    } catch (final IllegalArgumentException e) {
+      assertTrue("expected at least one null but got: scope="+scope+" category="+category+" metrics="+metrics,
+          (scope == null || category == null || metrics == null));
+      assertRegistered(scope, new HashMap<>(), coreMetricManager);
+    }
+  }
+
+  @Test
+  public void testRegisterMetricsWithReplacements() {
+    Random random = random();
+
+    Map<String, Counter> registered = new HashMap<>();
+    String scope = SolrMetricTestUtils.getRandomScope(random, true);
+    SolrInfoMBean.Category category = SolrMetricTestUtils.getRandomCategory(random, true);
+
+    int iterations = TestUtil.nextInt(random, 0, MAX_ITERATIONS);
+    for (int i = 0; i < iterations; ++i) {
+      Map<String, Counter> metrics = SolrMetricTestUtils.getRandomMetricsWithReplacements(random, registered);
+      if (metrics.isEmpty()) {
+        continue;
+      }
+      SolrMetricProducer producer = SolrMetricTestUtils.getProducerOf(metricManager, category, scope, metrics);
+      coreMetricManager.registerMetricProducer(scope, producer);
+      registered.putAll(metrics);
+      assertRegistered(scope, registered, coreMetricManager);
+    }
+  }
+
+  @Test
+  public void testLoadReporter() throws Exception {
+    Random random = random();
+
+    String className = MockMetricReporter.class.getName();
+    String reporterName = TestUtil.randomUnicodeString(random);
+
+    Map<String, Object> attrs = new HashMap<>();
+    attrs.put(FieldType.CLASS_NAME, className);
+    attrs.put(CoreAdminParams.NAME, reporterName);
+
+    boolean shouldDefineConfigurable = random.nextBoolean();
+    String configurable = TestUtil.randomUnicodeString(random);
+    if (shouldDefineConfigurable) attrs.put("configurable", configurable);
+
+    boolean shouldDefinePlugin = random.nextBoolean();
+    PluginInfo pluginInfo = shouldDefinePlugin ? new PluginInfo(TestUtil.randomUnicodeString(random), attrs) : null;
+
+    try {
+      metricManager.loadReporter(coreMetricManager.getRegistryName(), coreMetricManager.getCore().getResourceLoader(), pluginInfo);
+      assertNotNull(pluginInfo);
+      Map<String, SolrMetricReporter> reporters = metricManager.getReporters(coreMetricManager.getRegistryName());
+      assertTrue("reporters.size should be > 0, but was + " + reporters.size(), reporters.size() > 0);
+      assertNotNull("reporter " + reporterName + " not present among " + reporters, reporters.get(reporterName));
+      assertTrue("wrong reporter class: " + reporters.get(reporterName), reporters.get(reporterName) instanceof MockMetricReporter);
+    } catch (IllegalArgumentException e) {
+      assertTrue(pluginInfo == null || attrs.get("configurable") == null);
+      assertNull(metricManager.getReporters(coreMetricManager.getRegistryName()).get(reporterName));
+    }
+  }
+
+  private void assertRegistered(String scope, Map<String, Counter> newMetrics, SolrCoreMetricManager coreMetricManager) {
+    if (scope == null) {
+      return;
+    }
+    String filter = "." + scope + ".";
+    MetricRegistry registry = metricManager.registry(coreMetricManager.getRegistryName());
+    assertEquals(newMetrics.size(), registry.getMetrics().
+        keySet().stream().filter(s -> s.contains(filter)).count());
+
+    Map<String, Metric> registeredMetrics = registry.getMetrics().
+        entrySet().stream().filter(e -> e.getKey() != null && e.getKey().contains(filter)).
+        collect(Collectors.toMap(e -> e.getKey(), e -> e.getValue()));
+    for (Map.Entry<String, Metric> entry : registeredMetrics.entrySet()) {
+      String name = entry.getKey();
+      Metric expectedMetric = entry.getValue();
+
+      Metric actualMetric = registry.getMetrics().get(name);
+
+      assertNotNull(actualMetric);
+      assertEquals(expectedMetric, actualMetric);
+    }
+  }
+
+  @Test
+  public void testRegistryName() throws Exception {
+    String collectionName = "my_collection_";
+    String cloudCoreName = "my_collection__shard1_0_replica0";
+    String simpleCoreName = "collection_1_replica0";
+    String simpleRegistryName = "solr.core." + simpleCoreName;
+    String cloudRegistryName = "solr.core." + cloudCoreName;
+    String nestedRegistryName = "solr.core.my_collection_.shard1_0.replica0";
+    // pass through
+    assertEquals(cloudRegistryName, coreMetricManager.createRegistryName(null, cloudCoreName));
+    assertEquals(simpleRegistryName, coreMetricManager.createRegistryName(null, simpleCoreName));
+    // unknown naming scheme -> pass through
+    assertEquals(simpleRegistryName, coreMetricManager.createRegistryName(collectionName, simpleCoreName));
+    // cloud collection
+    assertEquals(nestedRegistryName, coreMetricManager.createRegistryName(collectionName, cloudCoreName));
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/test/org/apache/solr/metrics/SolrMetricManagerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/SolrMetricManagerTest.java b/solr/core/src/test/org/apache/solr/metrics/SolrMetricManagerTest.java
new file mode 100644
index 0000000..ecddfba
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/metrics/SolrMetricManagerTest.java
@@ -0,0 +1,273 @@
+/*
+ * 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;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricRegistry;
+import org.apache.lucene.util.TestUtil;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.core.SolrInfoMBean;
+import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.metrics.reporters.MockMetricReporter;
+import org.junit.Test;
+
+public class SolrMetricManagerTest extends SolrTestCaseJ4 {
+
+  @Test
+  public void testOverridableRegistryName() throws Exception {
+    Random r = random();
+    String originalName = TestUtil.randomSimpleString(r, 1, 10);
+    String targetName = TestUtil.randomSimpleString(r, 1, 10);
+    // no override
+    String result = SolrMetricManager.overridableRegistryName(originalName);
+    assertEquals(SolrMetricManager.REGISTRY_NAME_PREFIX + originalName, result);
+    // with override
+    System.setProperty(SolrMetricManager.REGISTRY_NAME_PREFIX + originalName, targetName);
+    result = SolrMetricManager.overridableRegistryName(originalName);
+    assertEquals(SolrMetricManager.REGISTRY_NAME_PREFIX + targetName, result);
+  }
+
+  @Test
+  public void testMoveMetrics() throws Exception {
+    Random r = random();
+
+    SolrMetricManager metricManager = new SolrMetricManager();
+
+    Map<String, Counter> metrics1 = SolrMetricTestUtils.getRandomMetrics(r, true);
+    Map<String, Counter> metrics2 = SolrMetricTestUtils.getRandomMetrics(r, true);
+    String fromName = TestUtil.randomSimpleString(r, 1, 10);
+    String toName = TestUtil.randomSimpleString(r, 1, 10);
+    // register test metrics
+    for (Map.Entry<String, Counter> entry : metrics1.entrySet()) {
+      metricManager.register(fromName, entry.getValue(), false, entry.getKey(), "metrics1");
+    }
+    for (Map.Entry<String, Counter> entry : metrics2.entrySet()) {
+      metricManager.register(fromName, entry.getValue(), false, entry.getKey(), "metrics2");
+    }
+    assertEquals(metrics1.size() + metrics2.size(), metricManager.registry(fromName).getMetrics().size());
+
+    // move metrics1
+    metricManager.moveMetrics(fromName, toName, new SolrMetricManager.PrefixFilter("metrics1"));
+    // check the remaining metrics
+    Map<String, Metric> fromMetrics = metricManager.registry(fromName).getMetrics();
+    assertEquals(metrics2.size(), fromMetrics.size());
+    for (Map.Entry<String, Counter> entry : metrics2.entrySet()) {
+      Object value = fromMetrics.get(SolrMetricManager.mkName(entry.getKey(), "metrics2"));
+      assertNotNull(value);
+      assertEquals(entry.getValue(), value);
+    }
+    // check the moved metrics
+    Map<String, Metric> toMetrics = metricManager.registry(toName).getMetrics();
+    assertEquals(metrics1.size(), toMetrics.size());
+    for (Map.Entry<String, Counter> entry : metrics1.entrySet()) {
+      Object value = toMetrics.get(SolrMetricManager.mkName(entry.getKey(), "metrics1"));
+      assertNotNull(value);
+      assertEquals(entry.getValue(), value);
+    }
+
+    // move all remaining metrics
+    metricManager.moveMetrics(fromName, toName, null);
+    fromMetrics = metricManager.registry(fromName).getMetrics();
+    assertEquals(0, fromMetrics.size());
+    toMetrics = metricManager.registry(toName).getMetrics();
+    assertEquals(metrics1.size() + metrics2.size(), toMetrics.size());
+  }
+
+  @Test
+  public void testRegisterAll() throws Exception {
+    Random r = random();
+
+    SolrMetricManager metricManager = new SolrMetricManager();
+
+    Map<String, Counter> metrics = SolrMetricTestUtils.getRandomMetrics(r, true);
+    MetricRegistry mr = new MetricRegistry();
+    for (Map.Entry<String, Counter> entry : metrics.entrySet()) {
+      mr.register(entry.getKey(), entry.getValue());
+    }
+
+    String registryName = TestUtil.randomSimpleString(r, 1, 10);
+    assertEquals(0, metricManager.registry(registryName).getMetrics().size());
+    metricManager.registerAll(registryName, mr, false);
+    // this should simply skip existing names
+    metricManager.registerAll(registryName, mr, true);
+    // this should produce error
+    try {
+      metricManager.registerAll(registryName, mr, false);
+      fail("registerAll with duplicate metric names should fail");
+    } catch (IllegalArgumentException e) {
+      // expected
+    }
+  }
+
+  @Test
+  public void testClearMetrics() throws Exception {
+    Random r = random();
+
+    SolrMetricManager metricManager = new SolrMetricManager();
+
+    Map<String, Counter> metrics = SolrMetricTestUtils.getRandomMetrics(r, true);
+    String registryName = TestUtil.randomSimpleString(r, 1, 10);
+
+    for (Map.Entry<String, Counter> entry : metrics.entrySet()) {
+      metricManager.register(registryName, entry.getValue(), false, entry.getKey(), "foo", "bar");
+    }
+    for (Map.Entry<String, Counter> entry : metrics.entrySet()) {
+      metricManager.register(registryName, entry.getValue(), false, entry.getKey(), "foo", "baz");
+    }
+    for (Map.Entry<String, Counter> entry : metrics.entrySet()) {
+      metricManager.register(registryName, entry.getValue(), false, entry.getKey(), "foo");
+    }
+
+    assertEquals(metrics.size() * 3, metricManager.registry(registryName).getMetrics().size());
+
+    // clear "foo.bar"
+    Set<String> removed = metricManager.clearMetrics(registryName, "foo", "bar");
+    assertEquals(metrics.size(), removed.size());
+    for (String s : removed) {
+      assertTrue(s.startsWith("foo.bar."));
+    }
+    removed = metricManager.clearMetrics(registryName, "foo", "baz");
+    assertEquals(metrics.size(), removed.size());
+    for (String s : removed) {
+      assertTrue(s.startsWith("foo.baz."));
+    }
+    // perhaps surprisingly, this works too - see PrefixFilter docs
+    removed = metricManager.clearMetrics(registryName, "fo");
+    assertEquals(metrics.size(), removed.size());
+    for (String s : removed) {
+      assertTrue(s.startsWith("foo."));
+    }
+  }
+
+  @Test
+  public void testSimpleMetrics() throws Exception {
+    Random r = random();
+
+    SolrMetricManager metricManager = new SolrMetricManager();
+
+    String registryName = TestUtil.randomSimpleString(r, 1, 10);
+
+    metricManager.counter(registryName, "simple_counter", "foo", "bar");
+    metricManager.timer(registryName, "simple_timer", "foo", "bar");
+    metricManager.meter(registryName, "simple_meter", "foo", "bar");
+    metricManager.histogram(registryName, "simple_histogram", "foo", "bar");
+    Map<String, Metric> metrics = metricManager.registry(registryName).getMetrics();
+    assertEquals(4, metrics.size());
+    for (Map.Entry<String, Metric> entry : metrics.entrySet()) {
+      assertTrue(entry.getKey().startsWith("foo.bar.simple_"));
+    }
+  }
+
+  @Test
+  public void testRegistryName() throws Exception {
+    Random r = random();
+
+    String name = TestUtil.randomSimpleString(r, 1, 10);
+
+    String result = SolrMetricManager.getRegistryName(SolrInfoMBean.Group.core, name, "collection1");
+    assertEquals("solr.core." + name + ".collection1", result);
+    // try it with already prefixed name - group will be ignored
+    result = SolrMetricManager.getRegistryName(SolrInfoMBean.Group.core, result);
+    assertEquals("solr.core." + name + ".collection1", result);
+    // try it with already prefixed name but with additional segments
+    result = SolrMetricManager.getRegistryName(SolrInfoMBean.Group.core, result, "shard1", "replica1");
+    assertEquals("solr.core." + name + ".collection1.shard1.replica1", result);
+  }
+
+  @Test
+  public void testReporters() throws Exception {
+    Random r = random();
+
+    SolrResourceLoader loader = new SolrResourceLoader();
+    SolrMetricManager metricManager = new SolrMetricManager();
+
+    PluginInfo[] plugins = new PluginInfo[] {
+        createPluginInfo("universal_foo", null, null),
+        createPluginInfo("multigroup_foo", "jvm, node, core", null),
+        createPluginInfo("multiregistry_foo", null, "solr.node, solr.core.collection1"),
+        createPluginInfo("specific_foo", null, "solr.core.collection1"),
+        createPluginInfo("node_foo", "node", null),
+        createPluginInfo("core_foo", "core", null)
+    };
+
+    metricManager.loadReporters(plugins, loader, SolrInfoMBean.Group.node);
+    Map<String, SolrMetricReporter> reporters = metricManager.getReporters(
+        SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node));
+    assertEquals(4, reporters.size());
+    assertTrue(reporters.containsKey("universal_foo"));
+    assertTrue(reporters.containsKey("multigroup_foo"));
+    assertTrue(reporters.containsKey("node_foo"));
+    assertTrue(reporters.containsKey("multiregistry_foo"));
+
+    metricManager.loadReporters(plugins, loader, SolrInfoMBean.Group.core, "collection1");
+    reporters = metricManager.getReporters(
+        SolrMetricManager.getRegistryName(SolrInfoMBean.Group.core, "collection1"));
+    assertEquals(5, reporters.size());
+    assertTrue(reporters.containsKey("universal_foo"));
+    assertTrue(reporters.containsKey("multigroup_foo"));
+    assertTrue(reporters.containsKey("specific_foo"));
+    assertTrue(reporters.containsKey("core_foo"));
+    assertTrue(reporters.containsKey("multiregistry_foo"));
+
+    metricManager.loadReporters(plugins, loader, SolrInfoMBean.Group.jvm);
+    reporters = metricManager.getReporters(
+        SolrMetricManager.getRegistryName(SolrInfoMBean.Group.jvm));
+    assertEquals(2, reporters.size());
+    assertTrue(reporters.containsKey("universal_foo"));
+    assertTrue(reporters.containsKey("multigroup_foo"));
+
+    metricManager.removeRegistry("solr.jvm");
+    reporters = metricManager.getReporters(
+        SolrMetricManager.getRegistryName(SolrInfoMBean.Group.jvm));
+    assertEquals(0, reporters.size());
+
+    metricManager.removeRegistry("solr.node");
+    reporters = metricManager.getReporters(
+        SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node));
+    assertEquals(0, reporters.size());
+
+    metricManager.removeRegistry("solr.core.collection1");
+    reporters = metricManager.getReporters(
+        SolrMetricManager.getRegistryName(SolrInfoMBean.Group.core, "collection1"));
+    assertEquals(0, reporters.size());
+
+  }
+
+  private PluginInfo createPluginInfo(String name, String group, String registry) {
+    Map<String,String> attrs = new HashMap<>();
+    attrs.put("name", name);
+    attrs.put("class", MockMetricReporter.class.getName());
+    if (group != null) {
+      attrs.put("group", group);
+    }
+    if (registry != null) {
+      attrs.put("registry", registry);
+    }
+    NamedList initArgs = new NamedList();
+    initArgs.add("configurable", "true");
+    return new PluginInfo("SolrMetricReporter", attrs, initArgs, null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/test/org/apache/solr/metrics/SolrMetricReporterTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/SolrMetricReporterTest.java b/solr/core/src/test/org/apache/solr/metrics/SolrMetricReporterTest.java
new file mode 100644
index 0000000..b275919
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/metrics/SolrMetricReporterTest.java
@@ -0,0 +1,69 @@
+/*
+ * 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;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+import org.apache.solr.common.params.CoreAdminParams;
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.metrics.reporters.MockMetricReporter;
+import org.apache.solr.schema.FieldType;
+import org.junit.Test;
+
+public class SolrMetricReporterTest extends LuceneTestCase {
+
+  @Test
+  public void testInit() throws Exception {
+    Random random = random();
+
+    SolrMetricManager metricManager = new SolrMetricManager();
+
+    final String registryName = TestUtil.randomSimpleString(random);
+    final MockMetricReporter reporter = new MockMetricReporter(metricManager, registryName);
+
+    Map<String, Object> attrs = new HashMap<>();
+    attrs.put(FieldType.CLASS_NAME, MockMetricReporter.class.getName());
+    attrs.put(CoreAdminParams.NAME, TestUtil.randomUnicodeString(random));
+
+    boolean shouldDefineConfigurable = random.nextBoolean();
+    String configurable = TestUtil.randomUnicodeString(random);
+    if (shouldDefineConfigurable) attrs.put("configurable", configurable);
+
+    boolean shouldDefinePlugin = random.nextBoolean();
+    String type = TestUtil.randomUnicodeString(random);
+    PluginInfo pluginInfo = shouldDefinePlugin ? new PluginInfo(type, attrs) : null;
+
+    try {
+      reporter.init(pluginInfo);
+      assertNotNull(pluginInfo);
+      assertEquals(configurable, attrs.get("configurable"));
+      assertTrue(reporter.didValidate);
+      assertNotNull(reporter.configurable);
+      assertEquals(configurable, reporter.configurable);
+    } catch (IllegalStateException e) {
+      assertTrue(pluginInfo == null || attrs.get("configurable") == null);
+      assertTrue(reporter.didValidate);
+      assertNull(reporter.configurable);
+    } finally {
+      reporter.close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/test/org/apache/solr/metrics/SolrMetricTestUtils.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/SolrMetricTestUtils.java b/solr/core/src/test/org/apache/solr/metrics/SolrMetricTestUtils.java
new file mode 100644
index 0000000..44ae9db
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/metrics/SolrMetricTestUtils.java
@@ -0,0 +1,140 @@
+/*
+ * 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;
+
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+
+import com.codahale.metrics.Counter;
+import org.apache.lucene.util.TestUtil;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.SolrInfoMBean;
+
+public final class SolrMetricTestUtils {
+
+  private static final int                    MAX_ITERATIONS = 100;
+  private static final SolrInfoMBean.Category CATEGORIES[]   = SolrInfoMBean.Category.values();
+
+  public static String getRandomScope(Random random) {
+    return getRandomScope(random, random.nextBoolean());
+  }
+
+  public static String getRandomScope(Random random, boolean shouldDefineScope) {
+    return shouldDefineScope ? TestUtil.randomSimpleString(random, 1, 10) : null; // must be simple string for JMX publishing
+  }
+
+  public static SolrInfoMBean.Category getRandomCategory(Random random) {
+    return getRandomCategory(random, random.nextBoolean());
+  }
+
+  public static SolrInfoMBean.Category getRandomCategory(Random random, boolean shouldDefineCategory) {
+    return shouldDefineCategory ? CATEGORIES[TestUtil.nextInt(random, 0, CATEGORIES.length - 1)] : null;
+  }
+
+  public static Map<String, Counter> getRandomMetrics(Random random) {
+    return getRandomMetrics(random, random.nextBoolean());
+  }
+
+  public static Map<String, Counter> getRandomMetrics(Random random, boolean shouldDefineMetrics) {
+    return shouldDefineMetrics ? getRandomMetricsWithReplacements(random, new HashMap<>()) : null;
+  }
+
+  public static final String SUFFIX = "_testing";
+
+  public static Map<String, Counter> getRandomMetricsWithReplacements(Random random, Map<String, Counter> existing) {
+    HashMap<String, Counter> metrics = new HashMap<>();
+    ArrayList<String> existingKeys = new ArrayList<>(existing.keySet());
+
+    int numMetrics = TestUtil.nextInt(random, 1, MAX_ITERATIONS);
+    for (int i = 0; i < numMetrics; ++i) {
+      boolean shouldReplaceMetric = !existing.isEmpty() && random.nextBoolean();
+      String name = shouldReplaceMetric
+          ? existingKeys.get(TestUtil.nextInt(random, 0, existingKeys.size() - 1))
+          : TestUtil.randomSimpleString(random, 1, 10) + SUFFIX; // must be simple string for JMX publishing
+
+      Counter counter = new Counter();
+      counter.inc(random.nextLong());
+      metrics.put(name, counter);
+    }
+
+    return metrics;
+  }
+
+  public static SolrMetricProducer getProducerOf(SolrMetricManager metricManager, SolrInfoMBean.Category category, String scope, Map<String, Counter> metrics) {
+    return new SolrMetricProducer() {
+      @Override
+      public Collection<String> initializeMetrics(SolrMetricManager manager, String registry, String scope) {
+        if (metrics == null || metrics.isEmpty()) {
+          return Collections.emptyList();
+        }
+        for (Map.Entry<String, Counter> entry : metrics.entrySet()) {
+          manager.counter(registry, entry.getKey(), category.toString(), scope);
+        }
+        return metrics.keySet();
+      }
+
+      @Override
+      public String getName() {
+        return scope;
+      }
+
+      @Override
+      public String getVersion() {
+        return "0.0";
+      }
+
+      @Override
+      public String getDescription() {
+        return "foo";
+      }
+
+      @Override
+      public Category getCategory() {
+        return category;
+      }
+
+      @Override
+      public String getSource() {
+        return null;
+      }
+
+      @Override
+      public URL[] getDocs() {
+        return new URL[0];
+      }
+
+      @Override
+      public NamedList getStatistics() {
+        return null;
+      }
+
+      @Override
+      public String toString() {
+        return "SolrMetricProducer.of{" +
+            "\ncategory=" + category +
+            "\nscope=" + scope +
+            "\nmetrics=" + metrics +
+            "\n}";
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java b/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java
new file mode 100644
index 0000000..c6449ac
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java
@@ -0,0 +1,143 @@
+/*
+ * 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;
+
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Random;
+
+import com.codahale.metrics.Timer;
+import org.apache.commons.io.FileUtils;
+import org.apache.lucene.util.TestUtil;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.NodeConfig;
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.core.SolrInfoMBean;
+import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.core.SolrXmlConfig;
+import org.apache.solr.metrics.reporters.MockMetricReporter;
+import org.apache.solr.util.TestHarness;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class SolrMetricsIntegrationTest extends SolrTestCaseJ4 {
+  private static final int MAX_ITERATIONS = 20;
+  private static final String CORE_NAME = "metrics_integration";
+  private static final String METRIC_NAME = "requestTimes";
+  private static final String HANDLER_NAME = "standard";
+  private static final String[] REPORTER_NAMES = {"reporter1", "reporter2"};
+  private static final String UNIVERSAL = "universal";
+  private static final String SPECIFIC = "specific";
+  private static final String MULTIGROUP = "multigroup";
+  private static final String MULTIREGISTRY = "multiregistry";
+  private static final String[] INITIAL_REPORTERS = {REPORTER_NAMES[0], REPORTER_NAMES[1], UNIVERSAL, SPECIFIC, MULTIGROUP, MULTIREGISTRY};
+  private static final String[] RENAMED_REPORTERS = {REPORTER_NAMES[0], REPORTER_NAMES[1], UNIVERSAL, MULTIGROUP};
+  private static final SolrInfoMBean.Category HANDLER_CATEGORY = SolrInfoMBean.Category.QUERYHANDLER;
+
+  private CoreContainer cc;
+  private SolrMetricManager metricManager;
+
+  @Before
+  public void beforeTest() throws Exception {
+    Path home = Paths.get(TEST_HOME());
+    // define these properties, they are used in solrconfig.xml
+    System.setProperty("solr.test.sys.prop1", "propone");
+    System.setProperty("solr.test.sys.prop2", "proptwo");
+    String solrXml = FileUtils.readFileToString(Paths.get(home.toString(), "solr-metricreporter.xml").toFile(), "UTF-8");
+    NodeConfig cfg = SolrXmlConfig.fromString(new SolrResourceLoader(home), solrXml);
+    cc = createCoreContainer(cfg,
+        new TestHarness.TestCoresLocator(DEFAULT_TEST_CORENAME, initCoreDataDir.getAbsolutePath(), "solrconfig.xml", "schema.xml"));
+    h.coreName = DEFAULT_TEST_CORENAME;
+    metricManager = cc.getMetricManager();
+    // initially there are more reporters, because two of them are added via a matching collection name
+    Map<String, SolrMetricReporter> reporters = metricManager.getReporters("solr.core." + DEFAULT_TEST_CORENAME);
+    assertEquals(INITIAL_REPORTERS.length, reporters.size());
+    assertTrue(reporters.keySet().containsAll(Arrays.asList(INITIAL_REPORTERS)));
+    // test rename operation
+    cc.rename(DEFAULT_TEST_CORENAME, CORE_NAME);
+    h.coreName = CORE_NAME;
+    cfg = cc.getConfig();
+    PluginInfo[] plugins = cfg.getMetricReporterPlugins();
+    assertNotNull(plugins);
+    assertEquals(10, plugins.length);
+    reporters = metricManager.getReporters("solr.node");
+    assertEquals(4, reporters.size());
+    assertTrue("Reporter '" + REPORTER_NAMES[0] + "' missing in solr.node", reporters.containsKey(REPORTER_NAMES[0]));
+    assertTrue("Reporter '" + UNIVERSAL + "' missing in solr.node", reporters.containsKey(UNIVERSAL));
+    assertTrue("Reporter '" + MULTIGROUP + "' missing in solr.node", reporters.containsKey(MULTIGROUP));
+    assertTrue("Reporter '" + MULTIREGISTRY + "' missing in solr.node", reporters.containsKey(MULTIREGISTRY));
+    SolrMetricReporter reporter = reporters.get(REPORTER_NAMES[0]);
+    assertTrue("Reporter " + reporter + " is not an instance of " + MockMetricReporter.class.getName(),
+        reporter instanceof  MockMetricReporter);
+    reporter = reporters.get(UNIVERSAL);
+    assertTrue("Reporter " + reporter + " is not an instance of " + MockMetricReporter.class.getName(),
+        reporter instanceof  MockMetricReporter);
+  }
+
+  @After
+  public void afterTest() throws Exception {
+    SolrCoreMetricManager coreMetricManager = h.getCore().getCoreMetricManager();
+    Map<String, SolrMetricReporter> reporters = metricManager.getReporters(coreMetricManager.getRegistryName());
+
+    deleteCore();
+
+    for (String reporterName : RENAMED_REPORTERS) {
+      SolrMetricReporter reporter = reporters.get(reporterName);
+      MockMetricReporter mockReporter = (MockMetricReporter) reporter;
+      assertTrue("Reporter " + reporterName + " was not closed: " + mockReporter, mockReporter.didClose);
+    }
+  }
+
+  @Test
+  public void testConfigureReporter() throws Exception {
+    Random random = random();
+
+    String metricName = SolrMetricManager.mkName(METRIC_NAME, HANDLER_CATEGORY.toString(), HANDLER_NAME);
+    SolrCoreMetricManager coreMetricManager = h.getCore().getCoreMetricManager();
+    Timer timer = (Timer) metricManager.timer(coreMetricManager.getRegistryName(), metricName);
+
+    long initialCount = timer.getCount();
+
+    int iterations = TestUtil.nextInt(random, 0, MAX_ITERATIONS);
+    for (int i = 0; i < iterations; ++i) {
+      h.query(req("*"));
+    }
+
+    long finalCount = timer.getCount();
+    assertEquals("metric counter incorrect", iterations, finalCount - initialCount);
+    Map<String, SolrMetricReporter> reporters = metricManager.getReporters(coreMetricManager.getRegistryName());
+    assertEquals(RENAMED_REPORTERS.length, reporters.size());
+
+    // SPECIFIC and MULTIREGISTRY were skipped because they were
+    // specific to collection1
+    for (String reporterName : RENAMED_REPORTERS) {
+      SolrMetricReporter reporter = reporters.get(reporterName);
+      assertNotNull("Reporter " + reporterName + " was not found.", reporter);
+      assertTrue(reporter instanceof MockMetricReporter);
+
+      MockMetricReporter mockReporter = (MockMetricReporter) reporter;
+      assertTrue("Reporter " + reporterName + " was not initialized: " + mockReporter, mockReporter.didInit);
+      assertTrue("Reporter " + reporterName + " was not validated: " + mockReporter, mockReporter.didValidate);
+      assertFalse("Reporter " + reporterName + " was incorrectly closed: " + mockReporter, mockReporter.didClose);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/test/org/apache/solr/metrics/reporters/MockMetricReporter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/reporters/MockMetricReporter.java b/solr/core/src/test/org/apache/solr/metrics/reporters/MockMetricReporter.java
new file mode 100644
index 0000000..2ecc33b
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/metrics/reporters/MockMetricReporter.java
@@ -0,0 +1,80 @@
+/*
+ * 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.reporters;
+
+import java.io.IOException;
+import java.util.Locale;
+import java.util.NoSuchElementException;
+
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricRegistry;
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.metrics.SolrMetricReporter;
+
+public class MockMetricReporter extends SolrMetricReporter {
+
+  public String configurable;
+
+  public boolean didInit = false;
+  public boolean didClose = false;
+  public boolean didValidate = false;
+
+  public MockMetricReporter(SolrMetricManager metricManager, String registryName) {
+    super(metricManager, registryName);
+  }
+
+  @Override
+  public void init(PluginInfo pluginInfo) {
+    super.init(pluginInfo);
+    didInit = true;
+  }
+
+  @Override
+  public void close() throws IOException {
+    didClose = true;
+  }
+
+  @Override
+  protected void validate() throws IllegalStateException {
+    didValidate = true;
+    if (configurable == null) {
+      throw new IllegalStateException("MockMetricReporter::configurable not defined.");
+    }
+  }
+
+  public void setConfigurable(String configurable) {
+    this.configurable = configurable;
+  }
+
+  public Metric reportMetric(String metricName) throws NoSuchElementException {
+    MetricRegistry registry = metricManager.registry(registryName);
+    Metric metric = registry.getMetrics().get(metricName);
+    if (metric == null) {
+      throw new NoSuchElementException("Metric was not found for metric name = " + metricName);
+    }
+
+    return metric;
+  }
+
+  @Override
+  public String toString() {
+    return String.format(Locale.ENGLISH, "[%s@%s: configurable = %s, didInit = %b, didValidate = %b, didClose = %b]",
+        getClass().getName(), Integer.toHexString(hashCode()), configurable, didInit, didValidate, didClose);
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/test/org/apache/solr/metrics/reporters/SolrJmxReporterTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/reporters/SolrJmxReporterTest.java b/solr/core/src/test/org/apache/solr/metrics/reporters/SolrJmxReporterTest.java
new file mode 100644
index 0000000..ea452b2
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/metrics/reporters/SolrJmxReporterTest.java
@@ -0,0 +1,156 @@
+/*
+ * 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.reporters;
+
+import javax.management.MBeanServer;
+import javax.management.ObjectInstance;
+import javax.management.ObjectName;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+
+import com.codahale.metrics.Counter;
+import org.apache.lucene.util.TestUtil;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.params.CoreAdminParams;
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.SolrInfoMBean;
+import org.apache.solr.metrics.SolrCoreMetricManager;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.metrics.SolrMetricProducer;
+import org.apache.solr.metrics.SolrMetricReporter;
+import org.apache.solr.metrics.SolrMetricTestUtils;
+import org.apache.solr.schema.FieldType;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class SolrJmxReporterTest extends SolrTestCaseJ4 {
+
+  private static final int MAX_ITERATIONS = 20;
+
+  private String domain;
+
+  private SolrCoreMetricManager coreMetricManager;
+  private SolrMetricManager metricManager;
+  private SolrJmxReporter reporter;
+  private MBeanServer mBeanServer;
+  private String reporterName;
+
+  @Before
+  public void beforeTest() throws Exception {
+    initCore("solrconfig-basic.xml", "schema.xml");
+
+    final SolrCore core = h.getCore();
+    domain = core.getName();
+
+    coreMetricManager = core.getCoreMetricManager();
+    metricManager = core.getCoreDescriptor().getCoreContainer().getMetricManager();
+    PluginInfo pluginInfo = createReporterPluginInfo();
+    metricManager.loadReporter(coreMetricManager.getRegistryName(), coreMetricManager.getCore().getResourceLoader(), pluginInfo);
+
+    Map<String, SolrMetricReporter> reporters = metricManager.getReporters(coreMetricManager.getRegistryName());
+    assertTrue("reporters.size should be > 0, but was + " + reporters.size(), reporters.size() > 0);
+    reporterName = pluginInfo.name;
+    assertNotNull("reporter " + reporterName + " not present among " + reporters, reporters.get(reporterName));
+    assertTrue("wrong reporter class: " + reporters.get(reporterName), reporters.get(reporterName) instanceof SolrJmxReporter);
+
+    reporter = (SolrJmxReporter) reporters.get(reporterName);
+    mBeanServer = reporter.getMBeanServer();
+    assertNotNull("MBean server not found.", mBeanServer);
+  }
+
+  private PluginInfo createReporterPluginInfo() {
+    Random random = random();
+    String className = SolrJmxReporter.class.getName();
+    String reporterName = TestUtil.randomSimpleString(random, 1, 10);
+
+    Map<String, Object> attrs = new HashMap<>();
+    attrs.put(FieldType.CLASS_NAME, className);
+    attrs.put(CoreAdminParams.NAME, reporterName);
+
+    boolean shouldOverrideDomain = random.nextBoolean();
+    if (shouldOverrideDomain) {
+      domain = TestUtil.randomSimpleString(random);
+      attrs.put("domain", domain);
+    }
+
+    return new PluginInfo(TestUtil.randomUnicodeString(random), attrs);
+  }
+
+  @After
+  public void afterTest() throws Exception {
+    metricManager.closeReporters(coreMetricManager.getRegistryName());
+    Set<ObjectInstance> objects =
+        mBeanServer.queryMBeans(ObjectName.getInstance(domain + ":*"), null);
+    assertTrue(objects.isEmpty());
+
+    coreMetricManager.close();
+    deleteCore();
+  }
+
+  @Test
+  public void testReportMetrics() throws Exception {
+    Random random = random();
+
+    Map<String, Counter> registered = new HashMap<>();
+    String scope = SolrMetricTestUtils.getRandomScope(random, true);
+    SolrInfoMBean.Category category = SolrMetricTestUtils.getRandomCategory(random, true);
+
+    int iterations = TestUtil.nextInt(random, 0, MAX_ITERATIONS);
+    for (int i = 0; i < iterations; ++i) {
+      Map<String, Counter> metrics = SolrMetricTestUtils.getRandomMetricsWithReplacements(random, registered);
+      SolrMetricProducer producer = SolrMetricTestUtils.getProducerOf(metricManager, category, scope, metrics);
+      coreMetricManager.registerMetricProducer(scope, producer);
+      registered.putAll(metrics);
+      //waitForListener();
+      Set<ObjectInstance> objects = mBeanServer.queryMBeans(null, null);
+      assertEquals(registered.size(), objects.stream().
+          filter(o -> scope.equals(o.getObjectName().getKeyProperty("scope")) &&
+                      reporterName.equals(o.getObjectName().getKeyProperty("reporter"))).count());
+    }
+  }
+
+  @Test
+  public void testReloadCore() throws Exception {
+    Random random = random();
+
+    String scope = SolrMetricTestUtils.getRandomScope(random, true);
+    SolrInfoMBean.Category category = SolrMetricTestUtils.getRandomCategory(random, true);
+    Map<String, Counter> metrics = SolrMetricTestUtils.getRandomMetrics(random, true);
+    SolrMetricProducer producer = SolrMetricTestUtils.getProducerOf(metricManager, category, scope, metrics);
+    coreMetricManager.registerMetricProducer(scope, producer);
+    Set<ObjectInstance> objects = mBeanServer.queryMBeans(null, null);
+    assertEquals(metrics.size(), objects.stream().
+        filter(o -> scope.equals(o.getObjectName().getKeyProperty("scope")) &&
+            reporterName.equals(o.getObjectName().getKeyProperty("reporter"))).count());
+
+    h.getCoreContainer().reload(h.getCore().getName());
+    PluginInfo pluginInfo = createReporterPluginInfo();
+    metricManager.loadReporter(coreMetricManager.getRegistryName(), coreMetricManager.getCore().getResourceLoader(), pluginInfo);
+    coreMetricManager.registerMetricProducer(scope, producer);
+
+    objects = mBeanServer.queryMBeans(null, null);
+    assertEquals(metrics.size(), objects.stream().
+        filter(o -> scope.equals(o.getObjectName().getKeyProperty("scope")) &&
+            pluginInfo.name.equals(o.getObjectName().getKeyProperty("reporter"))).count());
+  }
+
+}


[09/44] lucene-solr:jira/solr-9854: SOLR-8959: Refactored TestSegmentSorting out of TestMiniSolrCloudCluster

Posted by ab...@apache.org.
SOLR-8959: Refactored TestSegmentSorting out of TestMiniSolrCloudCluster


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

Branch: refs/heads/jira/solr-9854
Commit: 6525bb56f027655e5a01f028fa373305c0d01caa
Parents: 5123743
Author: Chris Hostetter <ho...@apache.org>
Authored: Wed Dec 14 13:18:56 2016 -0700
Committer: Chris Hostetter <ho...@apache.org>
Committed: Wed Dec 14 13:18:56 2016 -0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../cloud/SegmentTerminateEarlyTestState.java   |  12 +-
 .../solr/cloud/TestMiniSolrCloudCluster.java    |  50 -------
 .../apache/solr/cloud/TestSegmentSorting.java   | 133 +++++++++++++++++++
 4 files changed, 145 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6525bb56/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 5f0357b..73b0e9b 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -290,6 +290,9 @@ Other Changes
 
 * SOLR-9846: OverseerAutoReplicaFailoverThread can take too long to stop and leak out of unit tests. (Mark Miller)
 
+* SOLR-8959: Refactored TestSegmentSorting out of TestMiniSolrCloudCluster (hossman)
+
+
 ==================  6.3.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6525bb56/solr/core/src/test/org/apache/solr/cloud/SegmentTerminateEarlyTestState.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/SegmentTerminateEarlyTestState.java b/solr/core/src/test/org/apache/solr/cloud/SegmentTerminateEarlyTestState.java
index 199423b..b3df9e7 100644
--- a/solr/core/src/test/org/apache/solr/cloud/SegmentTerminateEarlyTestState.java
+++ b/solr/core/src/test/org/apache/solr/cloud/SegmentTerminateEarlyTestState.java
@@ -22,6 +22,7 @@ import java.time.ZonedDateTime;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
+import java.util.Random;
 
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
@@ -47,7 +48,12 @@ class SegmentTerminateEarlyTestState {
   Integer maxTimestampMM = null;
 
   int numDocs = 0;
+  final Random rand;
 
+  public SegmentTerminateEarlyTestState(Random rand) {
+    this.rand = rand;
+  }
+  
   void addDocuments(CloudSolrClient cloudSolrClient,
       int numCommits, int numDocsPerCommit, boolean optimize) throws Exception {
     for (int cc = 1; cc <= numCommits; ++cc) {
@@ -56,7 +62,7 @@ class SegmentTerminateEarlyTestState {
         final Integer docKey = new Integer(numDocs);
         SolrInputDocument doc = new SolrInputDocument();
         doc.setField(keyField, ""+docKey);
-        final int MM = TestMiniSolrCloudCluster.random().nextInt(60); // minutes
+        final int MM = rand.nextInt(60); // minutes
         if (minTimestampMM == null || MM <= minTimestampMM.intValue()) {
           if (minTimestampMM != null && MM < minTimestampMM.intValue()) {
             minTimestampDocKeys.clear();
@@ -116,7 +122,7 @@ class SegmentTerminateEarlyTestState {
     query.setFields(keyField, oddField, timestampField);
     final int rowsWanted = 1;
     query.setRows(rowsWanted);
-    final Boolean shardsInfoWanted = (TestMiniSolrCloudCluster.random().nextBoolean() ? null : new Boolean(TestMiniSolrCloudCluster.random().nextBoolean()));
+    final Boolean shardsInfoWanted = (rand.nextBoolean() ? null : new Boolean(rand.nextBoolean()));
     if (shardsInfoWanted != null) {
       query.set(ShardParams.SHARDS_INFO, shardsInfoWanted.booleanValue());
     }
@@ -163,7 +169,7 @@ class SegmentTerminateEarlyTestState {
     query.setSort(timestampField, SolrQuery.ORDER.desc);
     query.setFields(keyField, oddField, timestampField);
     query.setRows(1);
-    final Boolean shardsInfoWanted = (TestMiniSolrCloudCluster.random().nextBoolean() ? null : new Boolean(TestMiniSolrCloudCluster.random().nextBoolean()));
+    final Boolean shardsInfoWanted = (rand.nextBoolean() ? null : new Boolean(rand.nextBoolean()));
     if (shardsInfoWanted != null) {
       query.set(ShardParams.SHARDS_INFO, shardsInfoWanted.booleanValue());
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6525bb56/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudCluster.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudCluster.java b/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudCluster.java
index 97ecb67..de18875 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudCluster.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudCluster.java
@@ -16,7 +16,6 @@
  */
 package org.apache.solr.cloud;
 
-import java.io.File;
 import java.lang.invoke.MethodHandles;
 import java.net.URL;
 import java.util.ArrayList;
@@ -384,53 +383,4 @@ public class TestMiniSolrCloudCluster extends LuceneTestCase {
     }
   }
 
-  @Test
-  public void testSegmentTerminateEarly() throws Exception {
-
-    final String collectionName = "testSegmentTerminateEarlyCollection";
-
-    final SegmentTerminateEarlyTestState tstes = new SegmentTerminateEarlyTestState();
-
-    File solrXml = new File(SolrTestCaseJ4.TEST_HOME(), "solr.xml");
-    Builder jettyConfig = JettyConfig.builder();
-    jettyConfig.waitForLoadingCoresToFinish(null);
-    final MiniSolrCloudCluster miniCluster = createMiniSolrCloudCluster();
-    final CloudSolrClient cloudSolrClient = miniCluster.getSolrClient();
-    cloudSolrClient.setDefaultCollection(collectionName);
-
-    try {
-      // create collection
-      {
-        final String asyncId = (random().nextBoolean() ? null : "asyncId("+collectionName+".create)="+random().nextInt());
-        final Map<String, String> collectionProperties = new HashMap<>();
-        collectionProperties.put(CoreDescriptor.CORE_CONFIG, "solrconfig-sortingmergepolicyfactory.xml");
-        createCollection(miniCluster, collectionName, null, asyncId, Boolean.TRUE, collectionProperties);
-      }
-
-      ZkStateReader zkStateReader = cloudSolrClient.getZkStateReader();
-      AbstractDistribZkTestBase.waitForRecoveriesToFinish(collectionName, zkStateReader, true, true, 330);
-
-      // add some documents, then optimize to get merged-sorted segments
-      tstes.addDocuments(cloudSolrClient, 10, 10, true);
-
-      // CommonParams.SEGMENT_TERMINATE_EARLY parameter intentionally absent
-      tstes.queryTimestampDescending(cloudSolrClient);
-
-      // add a few more documents, but don't optimize to have some not-merge-sorted segments
-      tstes.addDocuments(cloudSolrClient, 2, 10, false);
-
-      // CommonParams.SEGMENT_TERMINATE_EARLY parameter now present
-      tstes.queryTimestampDescendingSegmentTerminateEarlyYes(cloudSolrClient);
-      tstes.queryTimestampDescendingSegmentTerminateEarlyNo(cloudSolrClient);
-
-      // CommonParams.SEGMENT_TERMINATE_EARLY parameter present but it won't be used
-      tstes.queryTimestampDescendingSegmentTerminateEarlyYesGrouped(cloudSolrClient);
-      tstes.queryTimestampAscendingSegmentTerminateEarlyYes(cloudSolrClient); // uses a sort order that is _not_ compatible with the merge sort order
-
-    }
-    finally {
-      miniCluster.shutdown();
-    }
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6525bb56/solr/core/src/test/org/apache/solr/cloud/TestSegmentSorting.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestSegmentSorting.java b/solr/core/src/test/org/apache/solr/cloud/TestSegmentSorting.java
new file mode 100644
index 0000000..016b63e
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/TestSegmentSorting.java
@@ -0,0 +1,133 @@
+/*
+ * 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 java.lang.invoke.MethodHandles;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.index.TieredMergePolicy;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.core.CoreDescriptor;
+import org.apache.solr.index.TieredMergePolicyFactory;
+
+import org.junit.After;
+import org.junit.BeforeClass;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestSegmentSorting extends SolrCloudTestCase {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  private static final int NUM_SERVERS = 5;
+  private static final int NUM_SHARDS = 2;
+  private static final int REPLICATION_FACTOR = 2;
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(NUM_SERVERS).configure();
+  }
+
+  @After
+  public void ensureClusterEmpty() throws Exception {
+    cluster.deleteAllCollections();
+    cluster.getSolrClient().setDefaultCollection(null);
+  }
+  
+  private void createCollection(MiniSolrCloudCluster miniCluster, String collectionName, String createNodeSet, String asyncId,
+      Boolean indexToPersist, Map<String,String> collectionProperties) throws Exception {
+    String configName = "solrCloudCollectionConfig";
+    miniCluster.uploadConfigSet(SolrTestCaseJ4.TEST_PATH().resolve("collection1").resolve("conf"), configName);
+
+    final boolean persistIndex = (indexToPersist != null ? indexToPersist.booleanValue() : random().nextBoolean());
+    if (collectionProperties == null) {
+      collectionProperties = new HashMap<>();
+    }
+    collectionProperties.putIfAbsent(CoreDescriptor.CORE_CONFIG, "solrconfig-tlog.xml");
+    collectionProperties.putIfAbsent("solr.tests.maxBufferedDocs", "100000");
+    collectionProperties.putIfAbsent("solr.tests.ramBufferSizeMB", "100");
+    // use non-test classes so RandomizedRunner isn't necessary
+    if (random().nextBoolean()) {
+      collectionProperties.putIfAbsent(SolrTestCaseJ4.SYSTEM_PROPERTY_SOLR_TESTS_MERGEPOLICY, TieredMergePolicy.class.getName());
+      collectionProperties.putIfAbsent(SolrTestCaseJ4.SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICY, "true");
+      collectionProperties.putIfAbsent(SolrTestCaseJ4.SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICYFACTORY, "false");
+    } else {
+      collectionProperties.putIfAbsent(SolrTestCaseJ4.SYSTEM_PROPERTY_SOLR_TESTS_MERGEPOLICYFACTORY, TieredMergePolicyFactory.class.getName());
+      collectionProperties.putIfAbsent(SolrTestCaseJ4.SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICYFACTORY, "true");
+      collectionProperties.putIfAbsent(SolrTestCaseJ4.SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICY, "false");
+    }
+    collectionProperties.putIfAbsent("solr.tests.mergeScheduler", "org.apache.lucene.index.ConcurrentMergeScheduler");
+    collectionProperties.putIfAbsent("solr.directoryFactory", (persistIndex ? "solr.StandardDirectoryFactory" : "solr.RAMDirectoryFactory"));
+
+    if (asyncId == null) {
+      CollectionAdminRequest.createCollection(collectionName, configName, NUM_SHARDS, REPLICATION_FACTOR)
+          .setCreateNodeSet(createNodeSet)
+          .setProperties(collectionProperties)
+          .process(miniCluster.getSolrClient());
+    }
+    else {
+      CollectionAdminRequest.createCollection(collectionName, configName, NUM_SHARDS, REPLICATION_FACTOR)
+          .setCreateNodeSet(createNodeSet)
+          .setProperties(collectionProperties)
+          .processAndWait(miniCluster.getSolrClient(), 30);
+    }
+  }
+
+
+  public void testSegmentTerminateEarly() throws Exception {
+
+    final String collectionName = "testSegmentTerminateEarlyCollection";
+
+    final SegmentTerminateEarlyTestState tstes = new SegmentTerminateEarlyTestState(random());
+    
+    final CloudSolrClient cloudSolrClient = cluster.getSolrClient();
+    cloudSolrClient.setDefaultCollection(collectionName);
+
+    // create collection
+    {
+      final String asyncId = (random().nextBoolean() ? null : "asyncId("+collectionName+".create)="+random().nextInt());
+      final Map<String, String> collectionProperties = new HashMap<>();
+      collectionProperties.put(CoreDescriptor.CORE_CONFIG, "solrconfig-sortingmergepolicyfactory.xml");
+      createCollection(cluster, collectionName, null, asyncId, Boolean.TRUE, collectionProperties);
+    }
+    
+    ZkStateReader zkStateReader = cloudSolrClient.getZkStateReader();
+    AbstractDistribZkTestBase.waitForRecoveriesToFinish(collectionName, zkStateReader, true, true, 330);
+    
+    // add some documents, then optimize to get merged-sorted segments
+    tstes.addDocuments(cloudSolrClient, 10, 10, true);
+    
+    // CommonParams.SEGMENT_TERMINATE_EARLY parameter intentionally absent
+    tstes.queryTimestampDescending(cloudSolrClient);
+    
+    // add a few more documents, but don't optimize to have some not-merge-sorted segments
+    tstes.addDocuments(cloudSolrClient, 2, 10, false);
+    
+    // CommonParams.SEGMENT_TERMINATE_EARLY parameter now present
+    tstes.queryTimestampDescendingSegmentTerminateEarlyYes(cloudSolrClient);
+    tstes.queryTimestampDescendingSegmentTerminateEarlyNo(cloudSolrClient);
+    
+    // CommonParams.SEGMENT_TERMINATE_EARLY parameter present but it won't be used
+    tstes.queryTimestampDescendingSegmentTerminateEarlyYesGrouped(cloudSolrClient);
+    tstes.queryTimestampAscendingSegmentTerminateEarlyYes(cloudSolrClient); // uses a sort order that is _not_ compatible with the merge sort order
+    
+  }
+}


[25/44] lucene-solr:jira/solr-9854: LUCENE-7590: make (Sorted)NumericDocValuesStats public

Posted by ab...@apache.org.
LUCENE-7590: make (Sorted)NumericDocValuesStats public


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

Branch: refs/heads/jira/solr-9854
Commit: 321c6f090f04463a8798d090e5426efeabbdc418
Parents: 23206ca
Author: Shai Erera <sh...@apache.org>
Authored: Mon Dec 19 10:14:58 2016 +0200
Committer: Shai Erera <sh...@apache.org>
Committed: Mon Dec 19 10:14:58 2016 +0200

----------------------------------------------------------------------
 .../src/java/org/apache/lucene/search/DocValuesStats.java | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/321c6f09/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java b/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
index b6449cc..f3319ee 100644
--- a/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
+++ b/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
@@ -44,7 +44,7 @@ public abstract class DocValuesStats<T> {
   }
 
   /**
-   * Called after #{@link DocValuesStats#accumulate(int)} was processed and verified that the document has a value for
+   * Called after {@link #accumulate(int)} was processed and verified that the document has a value for
    * the field. Implementations should update the statistics based on the value of the current document.
    *
    * @param count
@@ -89,18 +89,18 @@ public abstract class DocValuesStats<T> {
     return missing;
   }
 
-  /** The minimum value of the field. Undefined when {@link #count} is zero. */
+  /** The minimum value of the field. Undefined when {@link #count()} is zero. */
   public final T min() {
     return min;
   }
 
-  /** The maximum value of the field. Undefined when {@link #count} is zero. */
+  /** The maximum value of the field. Undefined when {@link #count()} is zero. */
   public final T max() {
     return max;
   }
 
   /** Holds statistics for a numeric DocValues field. */
-  static abstract class NumericDocValuesStats<T extends Number> extends DocValuesStats<T> {
+  public static abstract class NumericDocValuesStats<T extends Number> extends DocValuesStats<T> {
 
     protected double mean = 0.0;
     protected double variance = 0.0;
@@ -205,7 +205,7 @@ public abstract class DocValuesStats<T> {
   }
 
   /** Holds statistics for a sorted-numeric DocValues field. */
-  static abstract class SortedNumericDocValuesStats<T extends Number> extends DocValuesStats<T> {
+  public static abstract class SortedNumericDocValuesStats<T extends Number> extends DocValuesStats<T> {
 
     protected long valuesCount = 0;
     protected double mean = 0.0;


[35/44] lucene-solr:jira/solr-9854: SOLR-8542: techproducts example now includes (disabled) learning-to-rank support (enable via -Dsolr.ltr.enabled=true)

Posted by ab...@apache.org.
SOLR-8542: techproducts example now includes (disabled) learning-to-rank support (enable via -Dsolr.ltr.enabled=true)

additional changes as follows:

* LTRFeatureLoggerTransformerFactory:
** feature values cache name configurable (instead of hard-coded value that needs to match solrconfig.xml configuration)
** javadocs (example and parameters)

* CSV FeatureLogger:
** removed delimiter and separator assumptions in tests
** changed delimiter and separator (from "key:val;key:val" to "key=val,key=val")
** configurable (key value) delimiter and (features) separator

* JSON FeatureLogger:
** defer support for this (removing MapFeatureLogger class)

* adds 'training libraries' to (Linear|MultipleAdditiveTrees)Model javadocs

(Diego Ceccarelli, Michael Nilsson, Christine Poerschke)


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

Branch: refs/heads/jira/solr-9854
Commit: c8542b2bd0470af9f8d64bb8133f31828b342604
Parents: 0760944
Author: Christine Poerschke <cp...@apache.org>
Authored: Mon Dec 19 16:22:50 2016 +0000
Committer: Christine Poerschke <cp...@apache.org>
Committed: Mon Dec 19 16:22:50 2016 +0000

----------------------------------------------------------------------
 solr/contrib/ltr/README.md                      |   36 +-
 solr/contrib/ltr/example/solrconfig.xml         | 1711 ------------------
 .../org/apache/solr/ltr/CSVFeatureLogger.java   |   62 +
 .../java/org/apache/solr/ltr/FeatureLogger.java |  134 +-
 .../java/org/apache/solr/ltr/LTRRescorer.java   |    2 +-
 .../org/apache/solr/ltr/LTRScoringQuery.java    |    2 +-
 .../solr/ltr/SolrQueryRequestContextUtils.java  |    6 +-
 .../org/apache/solr/ltr/model/LinearModel.java  |   10 +
 .../ltr/model/MultipleAdditiveTreesModel.java   |    5 +
 .../LTRFeatureLoggerTransformerFactory.java     |  105 +-
 solr/contrib/ltr/src/java/overview.html         |    2 +-
 .../solr/collection1/conf/solrconfig-ltr.xml    |    5 +-
 .../collection1/conf/solrconfig-ltr_Th10_10.xml |    5 +-
 .../collection1/conf/solrconfig-multiseg.xml    |   12 +-
 .../apache/solr/ltr/FeatureLoggerTestUtils.java |   44 +
 .../org/apache/solr/ltr/TestLTROnSolrCloud.java |   17 +-
 .../solr/ltr/TestSelectiveWeightCreation.java   |   11 +-
 .../solr/ltr/feature/TestExternalFeatures.java  |   12 +-
 .../ltr/feature/TestExternalValueFeatures.java  |    4 +-
 .../solr/ltr/feature/TestFeatureLogging.java    |   99 +-
 .../solr/ltr/feature/TestFieldValueFeature.java |   13 +-
 .../solr/ltr/feature/TestFilterSolrFeature.java |    6 +-
 .../ltr/feature/TestNoMatchSolrFeature.java     |    6 +-
 .../ltr/feature/TestOriginalScoreFeature.java   |    9 +-
 .../conf/solrconfig.xml                         |   54 +-
 25 files changed, 379 insertions(+), 1993 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c8542b2b/solr/contrib/ltr/README.md
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/README.md b/solr/contrib/ltr/README.md
index 2033ffc..5c95056 100644
--- a/solr/contrib/ltr/README.md
+++ b/solr/contrib/ltr/README.md
@@ -13,6 +13,8 @@ deploy that model to Solr and use it to rerank your top X search results.
 
 Solr provides some simple example of indices. In order to test the plugin with
 the techproducts example please follow these steps.
+If you want to install the plugin on your instance of Solr, please refer
+to the [Solr Ref Guide](https://cwiki.apache.org/confluence/display/solr/Result+Reranking).
 
 1. Compile solr and the examples
 
@@ -20,38 +22,17 @@ the techproducts example please follow these steps.
     `ant dist`
     `ant server`
 
-2. Run the example to setup the index
+2. Run the example to setup the index, enabling the ltr plugin 
 
-   `./bin/solr -e techproducts`
+   `./bin/solr -e techproducts -Dsolr.ltr.enabled=true`
 
-3. Stop solr and install the plugin:
-     1. Stop solr
-
-        `./bin/solr stop`
-     2. Create the lib folder
-
-        `mkdir example/techproducts/solr/techproducts/lib`
-     3. Install the plugin in the lib folder
-
-        `cp build/contrib/ltr/solr-ltr-7.0.0-SNAPSHOT.jar example/techproducts/solr/techproducts/lib/`
-     4. Replace the original solrconfig with one importing all the ltr components
-
-        `cp contrib/ltr/example/solrconfig.xml example/techproducts/solr/techproducts/conf/`
-
-4. Run the example again
-
-   `./bin/solr -e techproducts`
-
-   Note you could also have just restarted your collection using the admin page.
-   You can find more detailed instructions [here](https://wiki.apache.org/solr/SolrPlugins).
-
-5. Deploy features and a model
+3. Deploy features and a model
 
       `curl -XPUT 'http://localhost:8983/solr/techproducts/schema/feature-store'  --data-binary "@./contrib/ltr/example/techproducts-features.json"  -H 'Content-type:application/json'`
 
       `curl -XPUT 'http://localhost:8983/solr/techproducts/schema/model-store'  --data-binary "@./contrib/ltr/example/techproducts-model.json"  -H 'Content-type:application/json'`
 
-6. Have fun !
+4. Have fun !
 
      * Access to the default feature store
 
@@ -99,7 +80,9 @@ BONUS: Train an actual machine learning model
   In order to get the feature vector you will have to
   specify that you want the field (e.g., fl="*,[features])  -->
 
-  <transformer name="features" class="org.apache.solr.ltr.response.transform.LTRFeatureLoggerTransformerFactory" />
+  <transformer name="features" class="org.apache.solr.ltr.response.transform.LTRFeatureLoggerTransformerFactory">
+    <str name="fvCacheName">QUERY_DOC_FV</str>
+  </transformer>
 
   <query>
     ...
@@ -395,6 +378,7 @@ About half the time for ranking is spent in the creation of weights for each fea
   
   <!-- Transformer for extracting features -->
   <transformer name="features" class="org.apache.solr.ltr.response.transform.LTRFeatureLoggerTransformerFactory">
+     <str name="fvCacheName">QUERY_DOC_FV</str>
      <int name="threadModule.totalPoolThreads">10</int> <!-- Maximum threads to share for all requests -->
      <int name="threadModule.numThreadsPerRequest">5</int> <!-- Maximum threads to use for a single request -->
   </transformer>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c8542b2b/solr/contrib/ltr/example/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/example/solrconfig.xml b/solr/contrib/ltr/example/solrconfig.xml
deleted file mode 100644
index 1c66c49..0000000
--- a/solr/contrib/ltr/example/solrconfig.xml
+++ /dev/null
@@ -1,1711 +0,0 @@
-<?xml version="1.0" encoding="UTF-8" ?>
-<!--
- 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.
--->
-
-<!--
-     For more details about configurations options that may appear in
-     this file, see http://wiki.apache.org/solr/SolrConfigXml.
--->
-<config>
-  <!-- In all configuration below, a prefix of "solr." for class names
-       is an alias that causes solr to search appropriate packages,
-       including org.apache.solr.(search|update|request|core|analysis)
-
-       You may also specify a fully qualified Java classname if you
-       have your own custom plugins.
-    -->
-
-  <!-- Controls what version of Lucene various components of Solr
-       adhere to.  Generally, you want to use the latest version to
-       get all bug fixes and improvements. It is highly recommended
-       that you fully re-index after changing this setting as it can
-       affect both how text is indexed and queried.
-  -->
-  <luceneMatchVersion>6.0.0</luceneMatchVersion>
-
-  <!-- <lib/> directives can be used to instruct Solr to load any Jars
-       identified and use them to resolve any "plugins" specified in
-       your solrconfig.xml or schema.xml (ie: Analyzers, Request
-       Handlers, etc...).
-
-       All directories and paths are resolved relative to the
-       instanceDir.
-
-       Please note that <lib/> directives are processed in the order
-       that they appear in your solrconfig.xml file, and are "stacked"
-       on top of each other when building a ClassLoader - so if you have
-       plugin jars with dependencies on other jars, the "lower level"
-       dependency jars should be loaded first.
-
-       If a "./lib" directory exists in your instanceDir, all files
-       found in it are included as if you had used the following
-       syntax...
-   -->
-              <lib dir="./lib" />
-
-
-  <!-- A 'dir' option by itself adds any files found in the directory
-       to the classpath, this is useful for including all jars in a
-       directory.
-
-       When a 'regex' is specified in addition to a 'dir', only the
-       files in that directory which completely match the regex
-       (anchored on both ends) will be included.
-
-       If a 'dir' option (with or without a regex) is used and nothing
-       is found that matches, a warning will be logged.
-
-       The examples below can be used to load some solr-contribs along
-       with their external dependencies.
-    -->
-  <lib dir="${solr.install.dir:../../../..}/contrib/extraction/lib" regex=".*\.jar" />
-  <lib dir="${solr.install.dir:../../../..}/dist/" regex="solr-cell-\d.*\.jar" />
-
-  <lib dir="${solr.install.dir:../../../..}/contrib/clustering/lib/" regex=".*\.jar" />
-  <lib dir="${solr.install.dir:../../../..}/dist/" regex="solr-clustering-\d.*\.jar" />
-
-  <lib dir="${solr.install.dir:../../../..}/contrib/langid/lib/" regex=".*\.jar" />
-  <lib dir="${solr.install.dir:../../../..}/dist/" regex="solr-langid-\d.*\.jar" />
-
-  <lib dir="${solr.install.dir:../../../..}/contrib/velocity/lib" regex=".*\.jar" />
-  <lib dir="${solr.install.dir:../../../..}/dist/" regex="solr-velocity-\d.*\.jar" />
-
-  <!-- an exact 'path' can be used instead of a 'dir' to specify a
-       specific jar file.  This will cause a serious error to be logged
-       if it can't be loaded.
-    -->
-  <!--
-     <lib path="../a-jar-that-does-not-exist.jar" />
-  -->
-
-  <!-- Data Directory
-
-       Used to specify an alternate directory to hold all index data
-       other than the default ./data under the Solr home.  If
-       replication is in use, this should match the replication
-       configuration.
-    -->
-  <dataDir>${solr.data.dir:}</dataDir>
-
-
-  <!-- The DirectoryFactory to use for indexes.
-
-       solr.StandardDirectoryFactory is filesystem
-       based and tries to pick the best implementation for the current
-       JVM and platform.  solr.NRTCachingDirectoryFactory, the default,
-       wraps solr.StandardDirectoryFactory and caches small files in memory
-       for better NRT performance.
-
-       One can force a particular implementation via solr.MMapDirectoryFactory,
-       solr.NIOFSDirectoryFactory, or solr.SimpleFSDirectoryFactory.
-
-       solr.RAMDirectoryFactory is memory based, not
-       persistent, and doesn't work with replication.
-    -->
-  <directoryFactory name="DirectoryFactory"
-                    class="${solr.directoryFactory:solr.NRTCachingDirectoryFactory}"/>
-
-  <!-- The CodecFactory for defining the format of the inverted index.
-       The default implementation is SchemaCodecFactory, which is the official Lucene
-       index format, but hooks into the schema to provide per-field customization of
-       the postings lists and per-document values in the fieldType element
-       (postingsFormat/docValuesFormat). Note that most of the alternative implementations
-       are experimental, so if you choose to customize the index format, it's a good
-       idea to convert back to the official format e.g. via IndexWriter.addIndexes(IndexReader)
-       before upgrading to a newer version to avoid unnecessary reindexing.
-       A "compressionMode" string element can be added to <codecFactory> to choose
-       between the existing compression modes in the default codec: "BEST_SPEED" (default)
-       or "BEST_COMPRESSION".
-  -->
-  <codecFactory class="solr.SchemaCodecFactory"/>
-
-  <!-- ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
-       Index Config - These settings control low-level behavior of indexing
-       Most example settings here show the default value, but are commented
-       out, to more easily see where customizations have been made.
-
-       Note: This replaces <indexDefaults> and <mainIndex> from older versions
-       ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ -->
-  <indexConfig>
-    <!-- maxFieldLength was removed in 4.0. To get similar behavior, include a
-         LimitTokenCountFilterFactory in your fieldType definition. E.g.
-     <filter class="solr.LimitTokenCountFilterFactory" maxTokenCount="10000"/>
-    -->
-    <!-- Maximum time to wait for a write lock (ms) for an IndexWriter. Default: 1000 -->
-    <!-- <writeLockTimeout>1000</writeLockTimeout>  -->
-
-    <!-- Expert: Enabling compound file will use less files for the index,
-         using fewer file descriptors on the expense of performance decrease.
-         Default in Lucene is "true". Default in Solr is "false" (since 3.6) -->
-    <!-- <useCompoundFile>false</useCompoundFile> -->
-
-    <!-- ramBufferSizeMB sets the amount of RAM that may be used by Lucene
-         indexing for buffering added documents and deletions before they are
-         flushed to the Directory.
-         maxBufferedDocs sets a limit on the number of documents buffered
-         before flushing.
-         If both ramBufferSizeMB and maxBufferedDocs is set, then
-         Lucene will flush based on whichever limit is hit first.
-         The default is 100 MB.  -->
-    <!-- <ramBufferSizeMB>100</ramBufferSizeMB> -->
-    <!-- <maxBufferedDocs>1000</maxBufferedDocs> -->
-
-    <!-- Expert: Merge Policy
-      -->
-    <!--
-        <mergePolicyFactory class="...">
-          ...
-        </mergePolicyFactory>
-      -->
-
-    <!-- Expert: Merge Scheduler
-         The Merge Scheduler in Lucene controls how merges are
-         performed.  The ConcurrentMergeScheduler (Lucene 2.3 default)
-         can perform merges in the background using separate threads.
-         The SerialMergeScheduler (Lucene 2.2 default) does not.
-     -->
-    <!--
-       <mergeScheduler class="org.apache.lucene.index.ConcurrentMergeScheduler"/>
-       -->
-
-    <!-- LockFactory
-
-         This option specifies which Lucene LockFactory implementation
-         to use.
-
-         single = SingleInstanceLockFactory - suggested for a
-                  read-only index or when there is no possibility of
-                  another process trying to modify the index.
-         native = NativeFSLockFactory - uses OS native file locking.
-                  Do not use when multiple solr webapps in the same
-                  JVM are attempting to share a single index.
-         simple = SimpleFSLockFactory  - uses a plain file for locking
-
-         Defaults: 'native' is default for Solr3.6 and later, otherwise
-                   'simple' is the default
-
-         More details on the nuances of each LockFactory...
-         http://wiki.apache.org/lucene-java/AvailableLockFactories
-    -->
-    <lockType>${solr.lock.type:native}</lockType>
-
-    <!-- Commit Deletion Policy
-         Custom deletion policies can be specified here. The class must
-         implement org.apache.lucene.index.IndexDeletionPolicy.
-
-         The default Solr IndexDeletionPolicy implementation supports
-         deleting index commit points on number of commits, age of
-         commit point and optimized status.
-
-         The latest commit point should always be preserved regardless
-         of the criteria.
-    -->
-    <!--
-    <deletionPolicy class="solr.SolrDeletionPolicy">
-    -->
-      <!-- The number of commit points to be kept -->
-      <!-- <str name="maxCommitsToKeep">1</str> -->
-      <!-- The number of optimized commit points to be kept -->
-      <!-- <str name="maxOptimizedCommitsToKeep">0</str> -->
-      <!--
-          Delete all commit points once they have reached the given age.
-          Supports DateMathParser syntax e.g.
-        -->
-      <!--
-         <str name="maxCommitAge">30MINUTES</str>
-         <str name="maxCommitAge">1DAY</str>
-      -->
-    <!--
-    </deletionPolicy>
-    -->
-
-    <!-- Lucene Infostream
-
-         To aid in advanced debugging, Lucene provides an "InfoStream"
-         of detailed information when indexing.
-
-         Setting the value to true will instruct the underlying Lucene
-         IndexWriter to write its info stream to solr's log. By default,
-         this is enabled here, and controlled through log4j.properties.
-      -->
-     <infoStream>true</infoStream>
-  </indexConfig>
-
-
-  <!-- JMX
-
-       This example enables JMX if and only if an existing MBeanServer
-       is found, use this if you want to configure JMX through JVM
-       parameters. Remove this to disable exposing Solr configuration
-       and statistics to JMX.
-
-       For more details see http://wiki.apache.org/solr/SolrJmx
-    -->
-  <jmx />
-  <!-- If you want to connect to a particular server, specify the
-       agentId
-    -->
-  <!-- <jmx agentId="myAgent" /> -->
-  <!-- If you want to start a new MBeanServer, specify the serviceUrl -->
-  <!-- <jmx serviceUrl="service:jmx:rmi:///jndi/rmi://localhost:9999/solr"/>
-    -->
-
-  <!-- The default high-performance update handler -->
-  <updateHandler class="solr.DirectUpdateHandler2">
-
-    <!-- Enables a transaction log, used for real-time get, durability, and
-         and solr cloud replica recovery.  The log can grow as big as
-         uncommitted changes to the index, so use of a hard autoCommit
-         is recommended (see below).
-         "dir" - the target directory for transaction logs, defaults to the
-                solr data directory.
-         "numVersionBuckets" - sets the number of buckets used to keep
-                track of max version values when checking for re-ordered
-                updates; increase this value to reduce the cost of
-                synchronizing access to version buckets during high-volume
-                indexing, this requires 8 bytes (long) * numVersionBuckets
-                of heap space per Solr core.
-    -->
-    <updateLog>
-      <str name="dir">${solr.ulog.dir:}</str>
-      <int name="numVersionBuckets">${solr.ulog.numVersionBuckets:65536}</int>
-    </updateLog>
-
-    <!-- AutoCommit
-
-         Perform a hard commit automatically under certain conditions.
-         Instead of enabling autoCommit, consider using "commitWithin"
-         when adding documents.
-
-         http://wiki.apache.org/solr/UpdateXmlMessages
-
-         maxDocs - Maximum number of documents to add since the last
-                   commit before automatically triggering a new commit.
-
-         maxTime - Maximum amount of time in ms that is allowed to pass
-                   since a document was added before automatically
-                   triggering a new commit.
-         openSearcher - if false, the commit causes recent index changes
-           to be flushed to stable storage, but does not cause a new
-           searcher to be opened to make those changes visible.
-
-         If the updateLog is enabled, then it's highly recommended to
-         have some sort of hard autoCommit to limit the log size.
-      -->
-     <autoCommit>
-       <maxTime>${solr.autoCommit.maxTime:15000}</maxTime>
-       <openSearcher>false</openSearcher>
-     </autoCommit>
-
-    <!-- softAutoCommit is like autoCommit except it causes a
-         'soft' commit which only ensures that changes are visible
-         but does not ensure that data is synced to disk.  This is
-         faster and more near-realtime friendly than a hard commit.
-      -->
-
-     <autoSoftCommit>
-       <maxTime>${solr.autoSoftCommit.maxTime:-1}</maxTime>
-     </autoSoftCommit>
-
-    <!-- Update Related Event Listeners
-
-         Various IndexWriter related events can trigger Listeners to
-         take actions.
-
-         postCommit - fired after every commit or optimize command
-         postOptimize - fired after every optimize command
-      -->
-    <!-- The RunExecutableListener executes an external command from a
-         hook such as postCommit or postOptimize.
-
-         exe - the name of the executable to run
-         dir - dir to use as the current working directory. (default=".")
-         wait - the calling thread waits until the executable returns.
-                (default="true")
-         args - the arguments to pass to the program.  (default is none)
-         env - environment variables to set.  (default is none)
-      -->
-    <!-- This example shows how RunExecutableListener could be used
-         with the script based replication...
-         http://wiki.apache.org/solr/CollectionDistribution
-      -->
-    <!--
-       <listener event="postCommit" class="solr.RunExecutableListener">
-         <str name="exe">solr/bin/snapshooter</str>
-         <str name="dir">.</str>
-         <bool name="wait">true</bool>
-         <arr name="args"> <str>arg1</str> <str>arg2</str> </arr>
-         <arr name="env"> <str>MYVAR=val1</str> </arr>
-       </listener>
-      -->
-
-  </updateHandler>
-
-  <!-- IndexReaderFactory
-
-       Use the following format to specify a custom IndexReaderFactory,
-       which allows for alternate IndexReader implementations.
-
-       ** Experimental Feature **
-
-       Please note - Using a custom IndexReaderFactory may prevent
-       certain other features from working. The API to
-       IndexReaderFactory may change without warning or may even be
-       removed from future releases if the problems cannot be
-       resolved.
-
-
-       ** Features that may not work with custom IndexReaderFactory **
-
-       The ReplicationHandler assumes a disk-resident index. Using a
-       custom IndexReader implementation may cause incompatibility
-       with ReplicationHandler and may cause replication to not work
-       correctly. See SOLR-1366 for details.
-
-    -->
-  <!--
-  <indexReaderFactory name="IndexReaderFactory" class="package.class">
-    <str name="someArg">Some Value</str>
-  </indexReaderFactory >
-  -->
-
-  <!-- ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
-       Query section - these settings control query time things like caches
-       ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ -->
-  <query>
-    <!-- Max Boolean Clauses
-
-         Maximum number of clauses in each BooleanQuery,  an exception
-         is thrown if exceeded.
-
-         ** WARNING **
-
-         This option actually modifies a global Lucene property that
-         will affect all SolrCores.  If multiple solrconfig.xml files
-         disagree on this property, the value at any given moment will
-         be based on the last SolrCore to be initialized.
-
-      -->
-    <maxBooleanClauses>1024</maxBooleanClauses>
-
-
-    <!-- Slow Query Threshold (in millis)
-
-         At high request rates, logging all requests can become a bottleneck
-         and therefore INFO logging is often turned off. However, it is still
-         useful to be able to set a latency threshold above which a request
-         is considered "slow" and log that request at WARN level so we can
-         easily identify slow queries.
-    -->
-    <slowQueryThresholdMillis>-1</slowQueryThresholdMillis>
-
-
-    <!-- Solr Internal Query Caches
-
-         There are two implementations of cache available for Solr,
-         LRUCache, based on a synchronized LinkedHashMap, and
-         FastLRUCache, based on a ConcurrentHashMap.
-
-         FastLRUCache has faster gets and slower puts in single
-         threaded operation and thus is generally faster than LRUCache
-         when the hit ratio of the cache is high (> 75%), and may be
-         faster under other scenarios on multi-cpu systems.
-    -->
-
-    <!-- Filter Cache
-
-         Cache used by SolrIndexSearcher for filters (DocSets),
-         unordered sets of *all* documents that match a query.  When a
-         new searcher is opened, its caches may be prepopulated or
-         "autowarmed" using data from caches in the old searcher.
-         autowarmCount is the number of items to prepopulate.  For
-         LRUCache, the autowarmed items will be the most recently
-         accessed items.
-
-         Parameters:
-           class - the SolrCache implementation LRUCache or
-               (LRUCache or FastLRUCache)
-           size - the maximum number of entries in the cache
-           initialSize - the initial capacity (number of entries) of
-               the cache.  (see java.util.HashMap)
-           autowarmCount - the number of entries to prepopulate from
-               and old cache.
-      -->
-    <filterCache class="solr.FastLRUCache"
-                 size="512"
-                 initialSize="512"
-                 autowarmCount="0"/>
-
-    <!-- Query Result Cache
-
-        Caches results of searches - ordered lists of document ids
-        (DocList) based on a query, a sort, and the range of documents requested.
-        Additional supported parameter by LRUCache:
-           maxRamMB - the maximum amount of RAM (in MB) that this cache is allowed
-                      to occupy
-     -->
-    <queryResultCache class="solr.LRUCache"
-                     size="512"
-                     initialSize="512"
-                     autowarmCount="0"/>
-
-    <!-- Document Cache
-
-         Caches Lucene Document objects (the stored fields for each
-         document).  Since Lucene internal document ids are transient,
-         this cache will not be autowarmed.
-      -->
-    <documentCache class="solr.LRUCache"
-                   size="512"
-                   initialSize="512"
-                   autowarmCount="0"/>
-
-    <!-- custom cache currently used by block join -->
-    <cache name="perSegFilter"
-      class="solr.search.LRUCache"
-      size="10"
-      initialSize="0"
-      autowarmCount="10"
-      regenerator="solr.NoOpRegenerator" />
-
-    <!-- Field Value Cache
-
-         Cache used to hold field values that are quickly accessible
-         by document id.  The fieldValueCache is created by default
-         even if not configured here.
-      -->
-    <!--
-       <fieldValueCache class="solr.FastLRUCache"
-                        size="512"
-                        autowarmCount="128"
-                        showItems="32" />
-      -->
-
-    <!-- Custom Cache
-
-         Example of a generic cache.  These caches may be accessed by
-         name through SolrIndexSearcher.getCache(),cacheLookup(), and
-         cacheInsert().  The purpose is to enable easy caching of
-         user/application level data.  The regenerator argument should
-         be specified as an implementation of solr.CacheRegenerator
-         if autowarming is desired.
-      -->
-      <!-- Cache for storing and fetching feature vectors -->
-    <cache name="QUERY_DOC_FV"
-      class="solr.search.LRUCache"
-      size="4096"
-      initialSize="2048"
-      autowarmCount="4096"
-      regenerator="solr.search.NoOpRegenerator" />
-    <!--
-       <cache name="myUserCache"
-              class="solr.LRUCache"
-              size="4096"
-              initialSize="1024"
-              autowarmCount="1024"
-              regenerator="com.mycompany.MyRegenerator"
-              />
-      -->
-
-
-    <!-- Lazy Field Loading
-
-         If true, stored fields that are not requested will be loaded
-         lazily.  This can result in a significant speed improvement
-         if the usual case is to not load all stored fields,
-         especially if the skipped fields are large compressed text
-         fields.
-    -->
-    <enableLazyFieldLoading>true</enableLazyFieldLoading>
-
-   <!-- Use Filter For Sorted Query
-
-        A possible optimization that attempts to use a filter to
-        satisfy a search.  If the requested sort does not include
-        score, then the filterCache will be checked for a filter
-        matching the query. If found, the filter will be used as the
-        source of document ids, and then the sort will be applied to
-        that.
-
-        For most situations, this will not be useful unless you
-        frequently get the same search repeatedly with different sort
-        options, and none of them ever use "score"
-     -->
-   <!--
-      <useFilterForSortedQuery>true</useFilterForSortedQuery>
-     -->
-
-   <!-- Result Window Size
-
-        An optimization for use with the queryResultCache.  When a search
-        is requested, a superset of the requested number of document ids
-        are collected.  For example, if a search for a particular query
-        requests matching documents 10 through 19, and queryWindowSize is 50,
-        then documents 0 through 49 will be collected and cached.  Any further
-        requests in that range can be satisfied via the cache.
-     -->
-   <queryResultWindowSize>20</queryResultWindowSize>
-
-   <!-- Maximum number of documents to cache for any entry in the
-        queryResultCache.
-     -->
-   <queryResultMaxDocsCached>200</queryResultMaxDocsCached>
-
-   <!-- Query Related Event Listeners
-
-        Various IndexSearcher related events can trigger Listeners to
-        take actions.
-
-        newSearcher - fired whenever a new searcher is being prepared
-        and there is a current searcher handling requests (aka
-        registered).  It can be used to prime certain caches to
-        prevent long request times for certain requests.
-
-        firstSearcher - fired whenever a new searcher is being
-        prepared but there is no current registered searcher to handle
-        requests or to gain autowarming data from.
-
-
-     -->
-    <!-- QuerySenderListener takes an array of NamedList and executes a
-         local query request for each NamedList in sequence.
-      -->
-    <listener event="newSearcher" class="solr.QuerySenderListener">
-      <arr name="queries">
-        <!--
-           <lst><str name="q">solr</str><str name="sort">price asc</str></lst>
-           <lst><str name="q">rocks</str><str name="sort">weight asc</str></lst>
-          -->
-      </arr>
-    </listener>
-    <listener event="firstSearcher" class="solr.QuerySenderListener">
-      <arr name="queries">
-        <lst>
-          <str name="q">static firstSearcher warming in solrconfig.xml</str>
-        </lst>
-      </arr>
-    </listener>
-
-    <!-- Use Cold Searcher
-
-         If a search request comes in and there is no current
-         registered searcher, then immediately register the still
-         warming searcher and use it.  If "false" then all requests
-         will block until the first searcher is done warming.
-      -->
-    <useColdSearcher>false</useColdSearcher>
-
-  </query>
-
-
-  <!-- Request Dispatcher
-
-       This section contains instructions for how the SolrDispatchFilter
-       should behave when processing requests for this SolrCore.
-
-       handleSelect is a legacy option that affects the behavior of requests
-       such as /select?qt=XXX
-
-       handleSelect="true" will cause the SolrDispatchFilter to process
-       the request and dispatch the query to a handler specified by the
-       "qt" param, assuming "/select" isn't already registered.
-
-       handleSelect="false" will cause the SolrDispatchFilter to
-       ignore "/select" requests, resulting in a 404 unless a handler
-       is explicitly registered with the name "/select"
-
-       handleSelect="true" is not recommended for new users, but is the default
-       for backwards compatibility
-    -->
-  <requestDispatcher handleSelect="false" >
-    <!-- Request Parsing
-
-         These settings indicate how Solr Requests may be parsed, and
-         what restrictions may be placed on the ContentStreams from
-         those requests
-
-         enableRemoteStreaming - enables use of the stream.file
-         and stream.url parameters for specifying remote streams.
-
-         multipartUploadLimitInKB - specifies the max size (in KiB) of
-         Multipart File Uploads that Solr will allow in a Request.
-
-         formdataUploadLimitInKB - specifies the max size (in KiB) of
-         form data (application/x-www-form-urlencoded) sent via
-         POST. You can use POST to pass request parameters not
-         fitting into the URL.
-
-         addHttpRequestToContext - if set to true, it will instruct
-         the requestParsers to include the original HttpServletRequest
-         object in the context map of the SolrQueryRequest under the
-         key "httpRequest". It will not be used by any of the existing
-         Solr components, but may be useful when developing custom
-         plugins.
-
-         *** WARNING ***
-         The settings below authorize Solr to fetch remote files, You
-         should make sure your system has some authentication before
-         using enableRemoteStreaming="true"
-
-      -->
-    <requestParsers enableRemoteStreaming="true"
-                    multipartUploadLimitInKB="2048000"
-                    formdataUploadLimitInKB="2048"
-                    addHttpRequestToContext="false"/>
-
-    <!-- HTTP Caching
-
-         Set HTTP caching related parameters (for proxy caches and clients).
-
-         The options below instruct Solr not to output any HTTP Caching
-         related headers
-      -->
-    <httpCaching never304="true" />
-    <!-- If you include a <cacheControl> directive, it will be used to
-         generate a Cache-Control header (as well as an Expires header
-         if the value contains "max-age=")
-
-         By default, no Cache-Control header is generated.
-
-         You can use the <cacheControl> option even if you have set
-         never304="true"
-      -->
-    <!--
-       <httpCaching never304="true" >
-         <cacheControl>max-age=30, public</cacheControl>
-       </httpCaching>
-      -->
-    <!-- To enable Solr to respond with automatically generated HTTP
-         Caching headers, and to response to Cache Validation requests
-         correctly, set the value of never304="false"
-
-         This will cause Solr to generate Last-Modified and ETag
-         headers based on the properties of the Index.
-
-         The following options can also be specified to affect the
-         values of these headers...
-
-         lastModFrom - the default value is "openTime" which means the
-         Last-Modified value (and validation against If-Modified-Since
-         requests) will all be relative to when the current Searcher
-         was opened.  You can change it to lastModFrom="dirLastMod" if
-         you want the value to exactly correspond to when the physical
-         index was last modified.
-
-         etagSeed="..." is an option you can change to force the ETag
-         header (and validation against If-None-Match requests) to be
-         different even if the index has not changed (ie: when making
-         significant changes to your config file)
-
-         (lastModifiedFrom and etagSeed are both ignored if you use
-         the never304="true" option)
-      -->
-    <!--
-       <httpCaching lastModifiedFrom="openTime"
-                    etagSeed="Solr">
-         <cacheControl>max-age=30, public</cacheControl>
-       </httpCaching>
-      -->
-  </requestDispatcher>
-
-  <!-- Request Handlers
-
-       http://wiki.apache.org/solr/SolrRequestHandler
-
-       Incoming queries will be dispatched to a specific handler by name
-       based on the path specified in the request.
-
-       Legacy behavior: If the request path uses "/select" but no Request
-       Handler has that name, and if handleSelect="true" has been specified in
-       the requestDispatcher, then the Request Handler is dispatched based on
-       the qt parameter.  Handlers without a leading '/' are accessed this way
-       like so: http://host/app/[core/]select?qt=name  If no qt is
-       given, then the requestHandler that declares default="true" will be
-       used or the one named "standard".
-
-       If a Request Handler is declared with startup="lazy", then it will
-       not be initialized until the first request that uses it.
-
-    -->
-  <!-- SearchHandler
-
-       http://wiki.apache.org/solr/SearchHandler
-
-       For processing Search Queries, the primary Request Handler
-       provided with Solr is "SearchHandler" It delegates to a sequent
-       of SearchComponents (see below) and supports distributed
-       queries across multiple shards
-    -->
-  <requestHandler name="/select" class="solr.SearchHandler">
-    <!-- default values for query parameters can be specified, these
-         will be overridden by parameters in the request
-      -->
-     <lst name="defaults">
-       <str name="echoParams">explicit</str>
-       <int name="rows">10</int>
-       <!-- Controls the distribution of a query to shards other than itself.
-            Consider making 'preferLocalShards' true when:
-              1) maxShardsPerNode > 1
-              2) Number of shards > 1
-              3) CloudSolrClient or LbHttpSolrServer is used by clients.
-            Without this option, every core broadcasts the distributed query to
-            a replica of each shard where the replicas are chosen randomly.
-            This option directs the cores to prefer cores hosted locally, thus
-            preventing network delays between machines.
-            This behavior also immunizes a bad/slow machine from slowing down all
-            the good machines (if those good machines were querying this bad one).
-
-            Specify this option=false for clients connecting through HttpSolrServer
-       -->
-       <bool name="preferLocalShards">false</bool>
-     </lst>
-    <!-- In addition to defaults, "appends" params can be specified
-         to identify values which should be appended to the list of
-         multi-val params from the query (or the existing "defaults").
-      -->
-    <!-- In this example, the param "fq=instock:true" would be appended to
-         any query time fq params the user may specify, as a mechanism for
-         partitioning the index, independent of any user selected filtering
-         that may also be desired (perhaps as a result of faceted searching).
-
-         NOTE: there is *absolutely* nothing a client can do to prevent these
-         "appends" values from being used, so don't use this mechanism
-         unless you are sure you always want it.
-      -->
-    <!--
-       <lst name="appends">
-         <str name="fq">inStock:true</str>
-       </lst>
-      -->
-    <!-- "invariants" are a way of letting the Solr maintainer lock down
-         the options available to Solr clients.  Any params values
-         specified here are used regardless of what values may be specified
-         in either the query, the "defaults", or the "appends" params.
-
-         In this example, the facet.field and facet.query params would
-         be fixed, limiting the facets clients can use.  Faceting is
-         not turned on by default - but if the client does specify
-         facet=true in the request, these are the only facets they
-         will be able to see counts for; regardless of what other
-         facet.field or facet.query params they may specify.
-
-         NOTE: there is *absolutely* nothing a client can do to prevent these
-         "invariants" values from being used, so don't use this mechanism
-         unless you are sure you always want it.
-      -->
-    <!--
-       <lst name="invariants">
-         <str name="facet.field">cat</str>
-         <str name="facet.field">manu_exact</str>
-         <str name="facet.query">price:[* TO 500]</str>
-         <str name="facet.query">price:[500 TO *]</str>
-       </lst>
-      -->
-    <!-- If the default list of SearchComponents is not desired, that
-         list can either be overridden completely, or components can be
-         prepended or appended to the default list.  (see below)
-      -->
-    <!--
-       <arr name="components">
-         <str>nameOfCustomComponent1</str>
-         <str>nameOfCustomComponent2</str>
-       </arr>
-      -->
-    </requestHandler>
-
- <!-- Query parser used to rerank top docs with a provided model -->
-  <queryParser name="ltr" class="org.apache.solr.ltr.search.LTRQParserPlugin" >
-    <int name="threadModule.totalPoolThreads">10</int> <!-- Maximum threads to use for all queries -->
-    <int name="threadModule.numThreadsPerRequest">10</int> <!-- Maximum threads to use for a single query-->
-  </queryParser>
-
-  <!--  Transformer that will encode the document features in the response. For each document the transformer
-  will add the features as an extra field in the response. The name of the field we will be the the name of the
-  transformer enclosed between brackets (in this case [features]). In order to get the feature vector you will have to
-  specify that you want the field (e.g., fl="*,[features])  -->
-  <transformer name="features" class="org.apache.solr.ltr.response.transform.LTRFeatureLoggerTransformerFactory" />
-
-
-  <!-- A request handler that returns indented JSON by default -->
-  <requestHandler name="/query" class="solr.SearchHandler">
-     <lst name="defaults">
-       <str name="echoParams">explicit</str>
-       <str name="wt">json</str>
-       <str name="indent">true</str>
-       <str name="df">text</str>
-     </lst>
- </requestHandler>
-
-
-  <!-- A Robust Example
-
-       This example SearchHandler declaration shows off usage of the
-       SearchHandler with many defaults declared
-
-       Note that multiple instances of the same Request Handler
-       (SearchHandler) can be registered multiple times with different
-       names (and different init parameters)
-    -->
-  <requestHandler name="/browse" class="solr.SearchHandler">
-     <lst name="defaults">
-       <str name="echoParams">explicit</str>
-
-       <!-- VelocityResponseWriter settings -->
-       <str name="wt">velocity</str>
-       <str name="v.template">browse</str>
-       <str name="v.layout">layout</str>
-       <str name="title">Solritas</str>
-
-       <!-- Query settings -->
-       <str name="defType">edismax</str>
-       <str name="qf">
-          text^0.5 features^1.0 name^1.2 sku^1.5 id^10.0 manu^1.1 cat^1.4
-          title^10.0 description^5.0 keywords^5.0 author^2.0 resourcename^1.0
-       </str>
-       <str name="mm">100%</str>
-       <str name="q.alt">*:*</str>
-       <str name="rows">10</str>
-       <str name="fl">*,score</str>
-
-       <str name="mlt.qf">
-         text^0.5 features^1.0 name^1.2 sku^1.5 id^10.0 manu^1.1 cat^1.4
-         title^10.0 description^5.0 keywords^5.0 author^2.0 resourcename^1.0
-       </str>
-       <str name="mlt.fl">text,features,name,sku,id,manu,cat,title,description,keywords,author,resourcename</str>
-       <int name="mlt.count">3</int>
-
-       <!-- Faceting defaults -->
-       <str name="facet">on</str>
-       <str name="facet.missing">true</str>
-       <str name="facet.field">cat</str>
-       <str name="facet.field">manu_exact</str>
-       <str name="facet.field">content_type</str>
-       <str name="facet.field">author_s</str>
-       <str name="facet.query">ipod</str>
-       <str name="facet.query">GB</str>
-       <str name="facet.mincount">1</str>
-       <str name="facet.pivot">cat,inStock</str>
-       <str name="facet.range.other">after</str>
-       <str name="facet.range">price</str>
-       <int name="f.price.facet.range.start">0</int>
-       <int name="f.price.facet.range.end">600</int>
-       <int name="f.price.facet.range.gap">50</int>
-       <str name="facet.range">popularity</str>
-       <int name="f.popularity.facet.range.start">0</int>
-       <int name="f.popularity.facet.range.end">10</int>
-       <int name="f.popularity.facet.range.gap">3</int>
-       <str name="facet.range">manufacturedate_dt</str>
-       <str name="f.manufacturedate_dt.facet.range.start">NOW/YEAR-10YEARS</str>
-       <str name="f.manufacturedate_dt.facet.range.end">NOW</str>
-       <str name="f.manufacturedate_dt.facet.range.gap">+1YEAR</str>
-       <str name="f.manufacturedate_dt.facet.range.other">before</str>
-       <str name="f.manufacturedate_dt.facet.range.other">after</str>
-
-       <!-- Highlighting defaults -->
-       <str name="hl">on</str>
-       <str name="hl.fl">content features title name</str>
-       <str name="hl.preserveMulti">true</str>
-       <str name="hl.encoder">html</str>
-       <str name="hl.simple.pre">&lt;b&gt;</str>
-       <str name="hl.simple.post">&lt;/b&gt;</str>
-       <str name="f.title.hl.fragsize">0</str>
-       <str name="f.title.hl.alternateField">title</str>
-       <str name="f.name.hl.fragsize">0</str>
-       <str name="f.name.hl.alternateField">name</str>
-       <str name="f.content.hl.snippets">3</str>
-       <str name="f.content.hl.fragsize">200</str>
-       <str name="f.content.hl.alternateField">content</str>
-       <str name="f.content.hl.maxAlternateFieldLength">750</str>
-
-       <!-- Spell checking defaults -->
-       <str name="spellcheck">on</str>
-       <str name="spellcheck.extendedResults">false</str>
-       <str name="spellcheck.count">5</str>
-       <str name="spellcheck.alternativeTermCount">2</str>
-       <str name="spellcheck.maxResultsForSuggest">5</str>
-       <str name="spellcheck.collate">true</str>
-       <str name="spellcheck.collateExtendedResults">true</str>
-       <str name="spellcheck.maxCollationTries">5</str>
-       <str name="spellcheck.maxCollations">3</str>
-     </lst>
-
-     <!-- append spellchecking to our list of components -->
-     <arr name="last-components">
-       <str>spellcheck</str>
-     </arr>
-  </requestHandler>
-
-
-  <initParams path="/update/**,/query,/select,/tvrh,/elevate,/spell,/browse">
-    <lst name="defaults">
-      <str name="df">text</str>
-    </lst>
-  </initParams>
-
-  <initParams path="/update/json/docs">
-    <lst name="defaults">
-      <!--this ensures that the entire json doc will be stored verbatim into one field-->
-      <str name="srcField">_src_</str>
-      <!--This means a the uniqueKeyField will be extracted from the fields and
-       all fields go into the 'df' field. In this config df is already configured to be 'text'
-        -->
-      <str name="mapUniqueKeyOnly">true</str>
-    </lst>
-
-  </initParams>
-
-  <!-- The following are implicitly added
-  <requestHandler name="/update/json" class="solr.UpdateRequestHandler">
-        <lst name="defaults">
-         <str name="stream.contentType">application/json</str>
-       </lst>
-  </requestHandler>
-  <requestHandler name="/update/csv" class="solr.UpdateRequestHandler">
-        <lst name="defaults">
-         <str name="stream.contentType">application/csv</str>
-       </lst>
-  </requestHandler>
-  -->
-
-  <!-- Solr Cell Update Request Handler
-
-       http://wiki.apache.org/solr/ExtractingRequestHandler
-
-    -->
-  <requestHandler name="/update/extract"
-                  startup="lazy"
-                  class="solr.extraction.ExtractingRequestHandler" >
-    <lst name="defaults">
-      <str name="lowernames">true</str>
-      <str name="uprefix">ignored_</str>
-
-      <!-- capture link hrefs but ignore div attributes -->
-      <str name="captureAttr">true</str>
-      <str name="fmap.a">links</str>
-      <str name="fmap.div">ignored_</str>
-    </lst>
-  </requestHandler>
-
-
-  <!-- Field Analysis Request Handler
-
-       RequestHandler that provides much the same functionality as
-       analysis.jsp. Provides the ability to specify multiple field
-       types and field names in the same request and outputs
-       index-time and query-time analysis for each of them.
-
-       Request parameters are:
-       analysis.fieldname - field name whose analyzers are to be used
-
-       analysis.fieldtype - field type whose analyzers are to be used
-       analysis.fieldvalue - text for index-time analysis
-       q (or analysis.q) - text for query time analysis
-       analysis.showmatch (true|false) - When set to true and when
-           query analysis is performed, the produced tokens of the
-           field value analysis will be marked as "matched" for every
-           token that is produces by the query analysis
-   -->
-  <requestHandler name="/analysis/field"
-                  startup="lazy"
-                  class="solr.FieldAnalysisRequestHandler" />
-
-
-  <!-- Document Analysis Handler
-
-       http://wiki.apache.org/solr/AnalysisRequestHandler
-
-       An analysis handler that provides a breakdown of the analysis
-       process of provided documents. This handler expects a (single)
-       content stream with the following format:
-
-       <docs>
-         <doc>
-           <field name="id">1</field>
-           <field name="name">The Name</field>
-           <field name="text">The Text Value</field>
-         </doc>
-         <doc>...</doc>
-         <doc>...</doc>
-         ...
-       </docs>
-
-    Note: Each document must contain a field which serves as the
-    unique key. This key is used in the returned response to associate
-    an analysis breakdown to the analyzed document.
-
-    Like the FieldAnalysisRequestHandler, this handler also supports
-    query analysis by sending either an "analysis.query" or "q"
-    request parameter that holds the query text to be analyzed. It
-    also supports the "analysis.showmatch" parameter which when set to
-    true, all field tokens that match the query tokens will be marked
-    as a "match".
-  -->
-  <requestHandler name="/analysis/document"
-                  class="solr.DocumentAnalysisRequestHandler"
-                  startup="lazy" />
-
-  <!-- Echo the request contents back to the client -->
-  <requestHandler name="/debug/dump" class="solr.DumpRequestHandler" >
-    <lst name="defaults">
-     <str name="echoParams">explicit</str>
-     <str name="echoHandler">true</str>
-    </lst>
-  </requestHandler>
-
-  <!-- Search Components
-
-       Search components are registered to SolrCore and used by
-       instances of SearchHandler (which can access them by name)
-
-       By default, the following components are available:
-
-       <searchComponent name="query"     class="solr.QueryComponent" />
-       <searchComponent name="facet"     class="solr.FacetComponent" />
-       <searchComponent name="mlt"       class="solr.MoreLikeThisComponent" />
-       <searchComponent name="highlight" class="solr.HighlightComponent" />
-       <searchComponent name="stats"     class="solr.StatsComponent" />
-       <searchComponent name="debug"     class="solr.DebugComponent" />
-
-       Default configuration in a requestHandler would look like:
-
-       <arr name="components">
-         <str>query</str>
-         <str>facet</str>
-         <str>mlt</str>
-         <str>highlight</str>
-         <str>stats</str>
-         <str>debug</str>
-       </arr>
-
-       If you register a searchComponent to one of the standard names,
-       that will be used instead of the default.
-
-       To insert components before or after the 'standard' components, use:
-
-       <arr name="first-components">
-         <str>myFirstComponentName</str>
-       </arr>
-
-       <arr name="last-components">
-         <str>myLastComponentName</str>
-       </arr>
-
-       NOTE: The component registered with the name "debug" will
-       always be executed after the "last-components"
-
-     -->
-
-   <!-- Spell Check
-
-        The spell check component can return a list of alternative spelling
-        suggestions.
-
-        http://wiki.apache.org/solr/SpellCheckComponent
-     -->
-  <searchComponent name="spellcheck" class="solr.SpellCheckComponent">
-
-    <str name="queryAnalyzerFieldType">text_general</str>
-
-    <!-- Multiple "Spell Checkers" can be declared and used by this
-         component
-      -->
-
-    <!-- a spellchecker built from a field of the main index -->
-    <lst name="spellchecker">
-      <str name="name">default</str>
-      <str name="field">text</str>
-      <str name="classname">solr.DirectSolrSpellChecker</str>
-      <!-- the spellcheck distance measure used, the default is the internal levenshtein -->
-      <str name="distanceMeasure">internal</str>
-      <!-- minimum accuracy needed to be considered a valid spellcheck suggestion -->
-      <float name="accuracy">0.5</float>
-      <!-- the maximum #edits we consider when enumerating terms: can be 1 or 2 -->
-      <int name="maxEdits">2</int>
-      <!-- the minimum shared prefix when enumerating terms -->
-      <int name="minPrefix">1</int>
-      <!-- maximum number of inspections per result. -->
-      <int name="maxInspections">5</int>
-      <!-- minimum length of a query term to be considered for correction -->
-      <int name="minQueryLength">4</int>
-      <!-- maximum threshold of documents a query term can appear to be considered for correction -->
-      <float name="maxQueryFrequency">0.01</float>
-      <!-- uncomment this to require suggestions to occur in 1% of the documents
-        <float name="thresholdTokenFrequency">.01</float>
-      -->
-    </lst>
-
-    <!-- a spellchecker that can break or combine words.  See "/spell" handler below for usage -->
-    <lst name="spellchecker">
-      <str name="name">wordbreak</str>
-      <str name="classname">solr.WordBreakSolrSpellChecker</str>
-      <str name="field">name</str>
-      <str name="combineWords">true</str>
-      <str name="breakWords">true</str>
-      <int name="maxChanges">10</int>
-    </lst>
-
-    <!-- a spellchecker that uses a different distance measure -->
-    <!--
-       <lst name="spellchecker">
-         <str name="name">jarowinkler</str>
-         <str name="field">spell</str>
-         <str name="classname">solr.DirectSolrSpellChecker</str>
-         <str name="distanceMeasure">
-           org.apache.lucene.search.spell.JaroWinklerDistance
-         </str>
-       </lst>
-     -->
-
-    <!-- a spellchecker that use an alternate comparator
-
-         comparatorClass be one of:
-          1. score (default)
-          2. freq (Frequency first, then score)
-          3. A fully qualified class name
-      -->
-    <!--
-       <lst name="spellchecker">
-         <str name="name">freq</str>
-         <str name="field">lowerfilt</str>
-         <str name="classname">solr.DirectSolrSpellChecker</str>
-         <str name="comparatorClass">freq</str>
-      -->
-
-    <!-- A spellchecker that reads the list of words from a file -->
-    <!--
-       <lst name="spellchecker">
-         <str name="classname">solr.FileBasedSpellChecker</str>
-         <str name="name">file</str>
-         <str name="sourceLocation">spellings.txt</str>
-         <str name="characterEncoding">UTF-8</str>
-         <str name="spellcheckIndexDir">spellcheckerFile</str>
-       </lst>
-      -->
-  </searchComponent>
-
-  <!-- A request handler for demonstrating the spellcheck component.
-
-       NOTE: This is purely as an example.  The whole purpose of the
-       SpellCheckComponent is to hook it into the request handler that
-       handles your normal user queries so that a separate request is
-       not needed to get suggestions.
-
-       IN OTHER WORDS, THERE IS REALLY GOOD CHANCE THE SETUP BELOW IS
-       NOT WHAT YOU WANT FOR YOUR PRODUCTION SYSTEM!
-
-       See http://wiki.apache.org/solr/SpellCheckComponent for details
-       on the request parameters.
-    -->
-  <requestHandler name="/spell" class="solr.SearchHandler" startup="lazy">
-    <lst name="defaults">
-      <!-- Solr will use suggestions from both the 'default' spellchecker
-           and from the 'wordbreak' spellchecker and combine them.
-           collations (re-written queries) can include a combination of
-           corrections from both spellcheckers -->
-      <str name="spellcheck.dictionary">default</str>
-      <str name="spellcheck.dictionary">wordbreak</str>
-      <str name="spellcheck">on</str>
-      <str name="spellcheck.extendedResults">true</str>
-      <str name="spellcheck.count">10</str>
-      <str name="spellcheck.alternativeTermCount">5</str>
-      <str name="spellcheck.maxResultsForSuggest">5</str>
-      <str name="spellcheck.collate">true</str>
-      <str name="spellcheck.collateExtendedResults">true</str>
-      <str name="spellcheck.maxCollationTries">10</str>
-      <str name="spellcheck.maxCollations">5</str>
-    </lst>
-    <arr name="last-components">
-      <str>spellcheck</str>
-    </arr>
-  </requestHandler>
-
-  <!-- The SuggestComponent in Solr provides users with automatic suggestions for query terms.
-       You can use this to implement a powerful auto-suggest feature in your search application.
-       As with the rest of this solrconfig.xml file, the configuration of this component is purely
-       an example that applies specifically to this configset and example documents.
-
-       More information about this component and other configuration options are described in the
-       "Suggester" section of the reference guide available at
-       http://archive.apache.org/dist/lucene/solr/ref-guide
-    -->
-  <searchComponent name="suggest" class="solr.SuggestComponent">
-    <lst name="suggester">
-      <str name="name">mySuggester</str>
-      <str name="lookupImpl">FuzzyLookupFactory</str>
-      <str name="dictionaryImpl">DocumentDictionaryFactory</str>
-      <str name="field">cat</str>
-      <str name="weightField">price</str>
-      <str name="suggestAnalyzerFieldType">string</str>
-      <str name="buildOnStartup">false</str>
-    </lst>
-  </searchComponent>
-
-  <requestHandler name="/suggest" class="solr.SearchHandler"
-                  startup="lazy" >
-    <lst name="defaults">
-      <str name="suggest">true</str>
-      <str name="suggest.count">10</str>
-    </lst>
-    <arr name="components">
-      <str>suggest</str>
-    </arr>
-  </requestHandler>
-
-
-  <!-- Term Vector Component
-
-       http://wiki.apache.org/solr/TermVectorComponent
-    -->
-  <searchComponent name="tvComponent" class="solr.TermVectorComponent"/>
-
-  <!-- A request handler for demonstrating the term vector component
-
-       This is purely as an example.
-
-       In reality you will likely want to add the component to your
-       already specified request handlers.
-    -->
-  <requestHandler name="/tvrh" class="solr.SearchHandler" startup="lazy">
-    <lst name="defaults">
-      <bool name="tv">true</bool>
-    </lst>
-    <arr name="last-components">
-      <str>tvComponent</str>
-    </arr>
-  </requestHandler>
-
-  <!-- Clustering Component
-
-       You'll need to set the solr.clustering.enabled system property
-       when running solr to run with clustering enabled:
-       -Dsolr.clustering.enabled=true
-
-       https://cwiki.apache.org/confluence/display/solr/Result+Clustering
-    -->
-  <searchComponent name="clustering"
-                   enable="${solr.clustering.enabled:false}"
-                   class="solr.clustering.ClusteringComponent" >
-    <!--
-    Declaration of "engines" (clustering algorithms).
-
-    The open source algorithms from Carrot2.org project:
-      * org.carrot2.clustering.lingo.LingoClusteringAlgorithm
-      * org.carrot2.clustering.stc.STCClusteringAlgorithm
-      * org.carrot2.clustering.kmeans.BisectingKMeansClusteringAlgorithm
-    See http://project.carrot2.org/algorithms.html for more information.
-
-    Commercial algorithm Lingo3G (needs to be installed separately):
-      * com.carrotsearch.lingo3g.Lingo3GClusteringAlgorithm
-    -->
-
-    <lst name="engine">
-      <str name="name">lingo3g</str>
-      <bool name="optional">true</bool>
-      <str name="carrot.algorithm">com.carrotsearch.lingo3g.Lingo3GClusteringAlgorithm</str>
-      <str name="carrot.resourcesDir">clustering/carrot2</str>
-    </lst>
-
-    <lst name="engine">
-      <str name="name">lingo</str>
-      <str name="carrot.algorithm">org.carrot2.clustering.lingo.LingoClusteringAlgorithm</str>
-      <str name="carrot.resourcesDir">clustering/carrot2</str>
-    </lst>
-
-    <lst name="engine">
-      <str name="name">stc</str>
-      <str name="carrot.algorithm">org.carrot2.clustering.stc.STCClusteringAlgorithm</str>
-      <str name="carrot.resourcesDir">clustering/carrot2</str>
-    </lst>
-
-    <lst name="engine">
-      <str name="name">kmeans</str>
-      <str name="carrot.algorithm">org.carrot2.clustering.kmeans.BisectingKMeansClusteringAlgorithm</str>
-      <str name="carrot.resourcesDir">clustering/carrot2</str>
-    </lst>
-  </searchComponent>
-
-  <!-- A request handler for demonstrating the clustering component.
-       This is meant as an example.
-       In reality you will likely want to add the component to your
-       already specified request handlers.
-    -->
-  <requestHandler name="/clustering"
-                  startup="lazy"
-                  enable="${solr.clustering.enabled:false}"
-                  class="solr.SearchHandler">
-    <lst name="defaults">
-      <bool name="clustering">true</bool>
-      <bool name="clustering.results">true</bool>
-      <!-- Field name with the logical "title" of a each document (optional) -->
-      <str name="carrot.title">name</str>
-      <!-- Field name with the logical "URL" of a each document (optional) -->
-      <str name="carrot.url">id</str>
-      <!-- Field name with the logical "content" of a each document (optional) -->
-      <str name="carrot.snippet">features</str>
-      <!-- Apply highlighter to the title/ content and use this for clustering. -->
-      <bool name="carrot.produceSummary">true</bool>
-      <!-- the maximum number of labels per cluster -->
-      <!--<int name="carrot.numDescriptions">5</int>-->
-      <!-- produce sub clusters -->
-      <bool name="carrot.outputSubClusters">false</bool>
-
-      <!-- Configure the remaining request handler parameters. -->
-      <str name="defType">edismax</str>
-      <str name="qf">
-        text^0.5 features^1.0 name^1.2 sku^1.5 id^10.0 manu^1.1 cat^1.4
-      </str>
-      <str name="q.alt">*:*</str>
-      <str name="rows">100</str>
-      <str name="fl">*,score</str>
-    </lst>
-    <arr name="last-components">
-      <str>clustering</str>
-    </arr>
-  </requestHandler>
-
-  <!-- Terms Component
-
-       http://wiki.apache.org/solr/TermsComponent
-
-       A component to return terms and document frequency of those
-       terms
-    -->
-  <searchComponent name="terms" class="solr.TermsComponent"/>
-
-  <!-- A request handler for demonstrating the terms component -->
-  <requestHandler name="/terms" class="solr.SearchHandler" startup="lazy">
-     <lst name="defaults">
-      <bool name="terms">true</bool>
-      <bool name="distrib">false</bool>
-    </lst>
-    <arr name="components">
-      <str>terms</str>
-    </arr>
-  </requestHandler>
-
-
-  <!-- Query Elevation Component
-
-       http://wiki.apache.org/solr/QueryElevationComponent
-
-       a search component that enables you to configure the top
-       results for a given query regardless of the normal lucene
-       scoring.
-    -->
-  <searchComponent name="elevator" class="solr.QueryElevationComponent" >
-    <!-- pick a fieldType to analyze queries -->
-    <str name="queryFieldType">string</str>
-    <str name="config-file">elevate.xml</str>
-  </searchComponent>
-
-  <!-- A request handler for demonstrating the elevator component -->
-  <requestHandler name="/elevate" class="solr.SearchHandler" startup="lazy">
-    <lst name="defaults">
-      <str name="echoParams">explicit</str>
-    </lst>
-    <arr name="last-components">
-      <str>elevator</str>
-    </arr>
-  </requestHandler>
-
-  <!-- Highlighting Component
-
-       http://wiki.apache.org/solr/HighlightingParameters
-    -->
-  <searchComponent class="solr.HighlightComponent" name="highlight">
-    <highlighting>
-      <!-- Configure the standard fragmenter -->
-      <!-- This could most likely be commented out in the "default" case -->
-      <fragmenter name="gap"
-                  default="true"
-                  class="solr.highlight.GapFragmenter">
-        <lst name="defaults">
-          <int name="hl.fragsize">100</int>
-        </lst>
-      </fragmenter>
-
-      <!-- A regular-expression-based fragmenter
-           (for sentence extraction)
-        -->
-      <fragmenter name="regex"
-                  class="solr.highlight.RegexFragmenter">
-        <lst name="defaults">
-          <!-- slightly smaller fragsizes work better because of slop -->
-          <int name="hl.fragsize">70</int>
-          <!-- allow 50% slop on fragment sizes -->
-          <float name="hl.regex.slop">0.5</float>
-          <!-- a basic sentence pattern -->
-          <str name="hl.regex.pattern">[-\w ,/\n\&quot;&apos;]{20,200}</str>
-        </lst>
-      </fragmenter>
-
-      <!-- Configure the standard formatter -->
-      <formatter name="html"
-                 default="true"
-                 class="solr.highlight.HtmlFormatter">
-        <lst name="defaults">
-          <str name="hl.simple.pre"><![CDATA[<em>]]></str>
-          <str name="hl.simple.post"><![CDATA[</em>]]></str>
-        </lst>
-      </formatter>
-
-      <!-- Configure the standard encoder -->
-      <encoder name="html"
-               class="solr.highlight.HtmlEncoder" />
-
-      <!-- Configure the standard fragListBuilder -->
-      <fragListBuilder name="simple"
-                       class="solr.highlight.SimpleFragListBuilder"/>
-
-      <!-- Configure the single fragListBuilder -->
-      <fragListBuilder name="single"
-                       class="solr.highlight.SingleFragListBuilder"/>
-
-      <!-- Configure the weighted fragListBuilder -->
-      <fragListBuilder name="weighted"
-                       default="true"
-                       class="solr.highlight.WeightedFragListBuilder"/>
-
-      <!-- default tag FragmentsBuilder -->
-      <fragmentsBuilder name="default"
-                        default="true"
-                        class="solr.highlight.ScoreOrderFragmentsBuilder">
-        <!--
-        <lst name="defaults">
-          <str name="hl.multiValuedSeparatorChar">/</str>
-        </lst>
-        -->
-      </fragmentsBuilder>
-
-      <!-- multi-colored tag FragmentsBuilder -->
-      <fragmentsBuilder name="colored"
-                        class="solr.highlight.ScoreOrderFragmentsBuilder">
-        <lst name="defaults">
-          <str name="hl.tag.pre"><![CDATA[
-               <b style="background:yellow">,<b style="background:lawgreen">,
-               <b style="background:aquamarine">,<b style="background:magenta">,
-               <b style="background:palegreen">,<b style="background:coral">,
-               <b style="background:wheat">,<b style="background:khaki">,
-               <b style="background:lime">,<b style="background:deepskyblue">]]></str>
-          <str name="hl.tag.post"><![CDATA[</b>]]></str>
-        </lst>
-      </fragmentsBuilder>
-
-      <boundaryScanner name="default"
-                       default="true"
-                       class="solr.highlight.SimpleBoundaryScanner">
-        <lst name="defaults">
-          <str name="hl.bs.maxScan">10</str>
-          <str name="hl.bs.chars">.,!? &#9;&#10;&#13;</str>
-        </lst>
-      </boundaryScanner>
-
-      <boundaryScanner name="breakIterator"
-                       class="solr.highlight.BreakIteratorBoundaryScanner">
-        <lst name="defaults">
-          <!-- type should be one of CHARACTER, WORD(default), LINE and SENTENCE -->
-          <str name="hl.bs.type">WORD</str>
-          <!-- language and country are used when constructing Locale object.  -->
-          <!-- And the Locale object will be used when getting instance of BreakIterator -->
-          <str name="hl.bs.language">en</str>
-          <str name="hl.bs.country">US</str>
-        </lst>
-      </boundaryScanner>
-    </highlighting>
-  </searchComponent>
-
-  <!-- Update Processors
-
-       Chains of Update Processor Factories for dealing with Update
-       Requests can be declared, and then used by name in Update
-       Request Processors
-
-       http://wiki.apache.org/solr/UpdateRequestProcessor
-
-    -->
-  <!-- Deduplication
-
-       An example dedup update processor that creates the "id" field
-       on the fly based on the hash code of some other fields.  This
-       example has overwriteDupes set to false since we are using the
-       id field as the signatureField and Solr will maintain
-       uniqueness based on that anyway.
-
-    -->
-  <!--
-     <updateRequestProcessorChain name="dedupe">
-       <processor class="solr.processor.SignatureUpdateProcessorFactory">
-         <bool name="enabled">true</bool>
-         <str name="signatureField">id</str>
-         <bool name="overwriteDupes">false</bool>
-         <str name="fields">name,features,cat</str>
-         <str name="signatureClass">solr.processor.Lookup3Signature</str>
-       </processor>
-       <processor class="solr.LogUpdateProcessorFactory" />
-       <processor class="solr.RunUpdateProcessorFactory" />
-     </updateRequestProcessorChain>
-    -->
-
-  <!-- Language identification
-
-       This example update chain identifies the language of the incoming
-       documents using the langid contrib. The detected language is
-       written to field language_s. No field name mapping is done.
-       The fields used for detection are text, title, subject and description,
-       making this example suitable for detecting languages form full-text
-       rich documents injected via ExtractingRequestHandler.
-       See more about langId at http://wiki.apache.org/solr/LanguageDetection
-    -->
-    <!--
-     <updateRequestProcessorChain name="langid">
-       <processor class="org.apache.solr.update.processor.TikaLanguageIdentifierUpdateProcessorFactory">
-         <str name="langid.fl">text,title,subject,description</str>
-         <str name="langid.langField">language_s</str>
-         <str name="langid.fallback">en</str>
-       </processor>
-       <processor class="solr.LogUpdateProcessorFactory" />
-       <processor class="solr.RunUpdateProcessorFactory" />
-     </updateRequestProcessorChain>
-    -->
-
-  <!-- Script update processor
-
-    This example hooks in an update processor implemented using JavaScript.
-
-    See more about the script update processor at http://wiki.apache.org/solr/ScriptUpdateProcessor
-  -->
-  <!--
-    <updateRequestProcessorChain name="script">
-      <processor class="solr.StatelessScriptUpdateProcessorFactory">
-        <str name="script">update-script.js</str>
-        <lst name="params">
-          <str name="config_param">example config parameter</str>
-        </lst>
-      </processor>
-      <processor class="solr.RunUpdateProcessorFactory" />
-    </updateRequestProcessorChain>
-  -->
-
-  <!-- Response Writers
-
-       http://wiki.apache.org/solr/QueryResponseWriter
-
-       Request responses will be written using the writer specified by
-       the 'wt' request parameter matching the name of a registered
-       writer.
-
-       The "default" writer is the default and will be used if 'wt' is
-       not specified in the request.
-    -->
-  <!-- The following response writers are implicitly configured unless
-       overridden...
-    -->
-  <!--
-     <queryResponseWriter name="xml"
-                          default="true"
-                          class="solr.XMLResponseWriter" />
-     <queryResponseWriter name="json" class="solr.JSONResponseWriter"/>
-     <queryResponseWriter name="python" class="solr.PythonResponseWriter"/>
-     <queryResponseWriter name="ruby" class="solr.RubyResponseWriter"/>
-     <queryResponseWriter name="php" class="solr.PHPResponseWriter"/>
-     <queryResponseWriter name="phps" class="solr.PHPSerializedResponseWriter"/>
-     <queryResponseWriter name="csv" class="solr.CSVResponseWriter"/>
-     <queryResponseWriter name="schema.xml" class="solr.SchemaXmlResponseWriter"/>
-    -->
-
-  <queryResponseWriter name="json" class="solr.JSONResponseWriter">
-     <!-- For the purposes of the tutorial, JSON responses are written as
-      plain text so that they are easy to read in *any* browser.
-      If you expect a MIME type of "application/json" just remove this override.
-     -->
-    <str name="content-type">text/plain; charset=UTF-8</str>
-  </queryResponseWriter>
-
-  <!--
-     Custom response writers can be declared as needed...
-    -->
-    <queryResponseWriter name="velocity" class="solr.VelocityResponseWriter" startup="lazy">
-      <str name="template.base.dir">${velocity.template.base.dir:}</str>
-    </queryResponseWriter>
-
-
-  <!-- XSLT response writer transforms the XML output by any xslt file found
-       in Solr's conf/xslt directory.  Changes to xslt files are checked for
-       every xsltCacheLifetimeSeconds.
-    -->
-  <queryResponseWriter name="xslt" class="solr.XSLTResponseWriter">
-    <int name="xsltCacheLifetimeSeconds">5</int>
-  </queryResponseWriter>
-
-  <!-- Query Parsers
-
-       http://wiki.apache.org/solr/SolrQuerySyntax
-
-       Multiple QParserPlugins can be registered by name, and then
-       used in either the "defType" param for the QueryComponent (used
-       by SearchHandler) or in LocalParams
-    -->
-  <!-- example of registering a query parser -->
-  <!--
-     <queryParser name="myparser" class="com.mycompany.MyQParserPlugin"/>
-    -->
-
-  <!-- Function Parsers
-
-       http://wiki.apache.org/solr/FunctionQuery
-
-       Multiple ValueSourceParsers can be registered by name, and then
-       used as function names when using the "func" QParser.
-    -->
-  <!-- example of registering a custom function parser  -->
-  <!--
-     <valueSourceParser name="myfunc"
-                        class="com.mycompany.MyValueSourceParser" />
-    -->
-
-
-  <!-- Document Transformers
-       http://wiki.apache.org/solr/DocTransformers
-    -->
-  <!--
-     Could be something like:
-     <transformer name="db" class="com.mycompany.LoadFromDatabaseTransformer" >
-       <int name="connection">jdbc://....</int>
-     </transformer>
-
-     To add a constant value to all docs, use:
-     <transformer name="mytrans2" class="org.apache.solr.response.transform.ValueAugmenterFactory" >
-       <int name="value">5</int>
-     </transformer>
-
-     If you want the user to still be able to change it with _value:something_ use this:
-     <transformer name="mytrans3" class="org.apache.solr.response.transform.ValueAugmenterFactory" >
-       <double name="defaultValue">5</double>
-     </transformer>
-
-      If you are using the QueryElevationComponent, you may wish to mark documents that get boosted.  The
-      EditorialMarkerFactory will do exactly that:
-     <transformer name="qecBooster" class="org.apache.solr.response.transform.EditorialMarkerFactory" />
-    -->
-
-
-  <!-- Legacy config for the admin interface -->
-  <admin>
-    <defaultQuery>*:*</defaultQuery>
-  </admin>
-
-</config>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c8542b2b/solr/contrib/ltr/src/java/org/apache/solr/ltr/CSVFeatureLogger.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/java/org/apache/solr/ltr/CSVFeatureLogger.java b/solr/contrib/ltr/src/java/org/apache/solr/ltr/CSVFeatureLogger.java
new file mode 100644
index 0000000..2da754e
--- /dev/null
+++ b/solr/contrib/ltr/src/java/org/apache/solr/ltr/CSVFeatureLogger.java
@@ -0,0 +1,62 @@
+/*
+ * 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.ltr;
+
+/**
+ * A feature logger that logs in csv format.
+ */
+public class CSVFeatureLogger extends FeatureLogger {
+  public static final char DEFAULT_KEY_VALUE_SEPARATOR = '=';
+  public static final char DEFAULT_FEATURE_SEPARATOR = ',';
+  private final char keyValueSep;
+  private final char featureSep;
+
+  public CSVFeatureLogger(String fvCacheName, FeatureFormat f) {
+    super(fvCacheName, f);
+    this.keyValueSep = DEFAULT_KEY_VALUE_SEPARATOR;
+    this.featureSep = DEFAULT_FEATURE_SEPARATOR;
+  }
+
+  public CSVFeatureLogger(String fvCacheName, FeatureFormat f, char keyValueSep, char featureSep) {
+    super(fvCacheName, f);
+    this.keyValueSep = keyValueSep;
+    this.featureSep = featureSep;
+  }
+
+  @Override
+  public String makeFeatureVector(LTRScoringQuery.FeatureInfo[] featuresInfo) {
+    // Allocate the buffer to a size based on the number of features instead of the 
+    // default 16.  You need space for the name, value, and two separators per feature, 
+    // but not all the features are expected to fire, so this is just a naive estimate. 
+    StringBuilder sb = new StringBuilder(featuresInfo.length * 3);
+    boolean isDense = featureFormat.equals(FeatureFormat.DENSE);
+    for (LTRScoringQuery.FeatureInfo featInfo:featuresInfo) {
+      if (featInfo.isUsed() || isDense){
+        sb.append(featInfo.getName())
+        .append(keyValueSep)
+        .append(featInfo.getValue())
+        .append(featureSep);
+      }
+    }
+
+    final String features = (sb.length() > 0 ? 
+        sb.substring(0, sb.length() - 1) : "");
+
+    return features;
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c8542b2b/solr/contrib/ltr/src/java/org/apache/solr/ltr/FeatureLogger.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/java/org/apache/solr/ltr/FeatureLogger.java b/solr/contrib/ltr/src/java/org/apache/solr/ltr/FeatureLogger.java
index 9c10c2c..ec14ab9 100644
--- a/solr/contrib/ltr/src/java/org/apache/solr/ltr/FeatureLogger.java
+++ b/solr/contrib/ltr/src/java/org/apache/solr/ltr/FeatureLogger.java
@@ -16,30 +16,22 @@
  */
 package org.apache.solr.ltr;
 
-import java.lang.invoke.MethodHandles;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
 import org.apache.solr.search.SolrIndexSearcher;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * FeatureLogger can be registered in a model and provide a strategy for logging
  * the feature values.
  */
-public abstract class FeatureLogger<FV_TYPE> {
-
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+public abstract class FeatureLogger {
 
   /** the name of the cache using for storing the feature value **/
-  private static final String QUERY_FV_CACHE_NAME = "QUERY_DOC_FV";
+  private final String fvCacheName;
 
-  protected enum FeatureFormat {DENSE, SPARSE};
+  public enum FeatureFormat {DENSE, SPARSE};
   protected final FeatureFormat featureFormat;
 
-  protected FeatureLogger(FeatureFormat f) {
+  protected FeatureLogger(String fvCacheName, FeatureFormat f) {
+    this.fvCacheName = fvCacheName;
     this.featureFormat = f;
   }
 
@@ -58,56 +50,16 @@ public abstract class FeatureLogger<FV_TYPE> {
 
   public boolean log(int docid, LTRScoringQuery scoringQuery,
       SolrIndexSearcher searcher, LTRScoringQuery.FeatureInfo[] featuresInfo) {
-    final FV_TYPE featureVector = makeFeatureVector(featuresInfo);
+    final String featureVector = makeFeatureVector(featuresInfo);
     if (featureVector == null) {
       return false;
     }
 
-    return searcher.cacheInsert(QUERY_FV_CACHE_NAME,
+    return searcher.cacheInsert(fvCacheName,
         fvCacheKey(scoringQuery, docid), featureVector) != null;
   }
 
-  /**
-   * returns a FeatureLogger that logs the features in output, using the format
-   * specified in the 'stringFormat' param: 'csv' will log the features as a unique
-   * string in csv format 'json' will log the features in a map in a Map of
-   * featureName keys to featureValue values if format is null or empty, csv
-   * format will be selected.
-   * 'featureFormat' param: 'dense' will write features in dense format,
-   * 'sparse' will write the features in sparse format, null or empty will
-   * default to 'sparse'
-   *
-   *
-   * @return a feature logger for the format specified.
-   */
-  public static FeatureLogger<?> createFeatureLogger(String stringFormat, String featureFormat) {
-    final FeatureFormat f;
-    if (featureFormat == null || featureFormat.isEmpty() ||
-        featureFormat.equals("sparse")) {
-      f = FeatureFormat.SPARSE;
-    }
-    else if (featureFormat.equals("dense")) {
-      f = FeatureFormat.DENSE;
-    }
-    else {
-      f = FeatureFormat.SPARSE;
-      log.warn("unknown feature logger feature format {} | {}", stringFormat, featureFormat);
-    }
-    if ((stringFormat == null) || stringFormat.isEmpty()) {
-      return new CSVFeatureLogger(f);
-    }
-    if (stringFormat.equals("csv")) {
-      return new CSVFeatureLogger(f);
-    }
-    if (stringFormat.equals("json")) {
-      return new MapFeatureLogger(f);
-    }
-    log.warn("unknown feature logger string format {} | {}", stringFormat, featureFormat);
-    return null;
-
-  }
-
-  public abstract FV_TYPE makeFeatureVector(LTRScoringQuery.FeatureInfo[] featuresInfo);
+  public abstract String makeFeatureVector(LTRScoringQuery.FeatureInfo[] featuresInfo);
 
   private static int fvCacheKey(LTRScoringQuery scoringQuery, int docid) {
     return  scoringQuery.hashCode() + (31 * docid);
@@ -121,75 +73,9 @@ public abstract class FeatureLogger<FV_TYPE> {
    * @return String representation of the list of features calculated for docid
    */
 
-  public FV_TYPE getFeatureVector(int docid, LTRScoringQuery scoringQuery,
+  public String getFeatureVector(int docid, LTRScoringQuery scoringQuery,
       SolrIndexSearcher searcher) {
-    return (FV_TYPE) searcher.cacheLookup(QUERY_FV_CACHE_NAME, fvCacheKey(scoringQuery, docid));
-  }
-
-
-  public static class MapFeatureLogger extends FeatureLogger<Map<String,Float>> {
-
-    public MapFeatureLogger(FeatureFormat f) {
-      super(f);
-    }
-
-    @Override
-    public Map<String,Float> makeFeatureVector(LTRScoringQuery.FeatureInfo[] featuresInfo) {
-      boolean isDense = featureFormat.equals(FeatureFormat.DENSE);
-      Map<String,Float> hashmap = Collections.emptyMap();
-      if (featuresInfo.length > 0) {
-        hashmap = new HashMap<String,Float>(featuresInfo.length);
-        for (LTRScoringQuery.FeatureInfo featInfo:featuresInfo){
-          if (featInfo.isUsed() || isDense){
-            hashmap.put(featInfo.getName(), featInfo.getValue());
-          }
-        }
-      }
-      return hashmap;
-    }
-
-  }
-
-  public static class CSVFeatureLogger extends FeatureLogger<String> {
-    char keyValueSep = ':';
-    char featureSep = ';';
-
-    public CSVFeatureLogger(FeatureFormat f) {
-      super(f);
-    }
-
-    public CSVFeatureLogger setKeyValueSep(char keyValueSep) {
-      this.keyValueSep = keyValueSep;
-      return this;
-    }
-
-    public CSVFeatureLogger setFeatureSep(char featureSep) {
-      this.featureSep = featureSep;
-      return this;
-    }
-
-    @Override
-    public String makeFeatureVector(LTRScoringQuery.FeatureInfo[] featuresInfo) {
-      // Allocate the buffer to a size based on the number of features instead of the
-      // default 16.  You need space for the name, value, and two separators per feature,
-      // but not all the features are expected to fire, so this is just a naive estimate.
-      StringBuilder sb = new StringBuilder(featuresInfo.length * 3);
-      boolean isDense = featureFormat.equals(FeatureFormat.DENSE);
-      for (LTRScoringQuery.FeatureInfo featInfo:featuresInfo) {
-        if (featInfo.isUsed() || isDense){
-          sb.append(featInfo.getName())
-          .append(keyValueSep)
-          .append(featInfo.getValue())
-          .append(featureSep);
-        }
-      }
-
-      final String features = (sb.length() > 0 ?
-          sb.substring(0, sb.length() - 1) : "");
-
-      return features;
-    }
-
+    return (String) searcher.cacheLookup(fvCacheName, fvCacheKey(scoringQuery, docid));
   }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c8542b2b/solr/contrib/ltr/src/java/org/apache/solr/ltr/LTRRescorer.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/java/org/apache/solr/ltr/LTRRescorer.java b/solr/contrib/ltr/src/java/org/apache/solr/ltr/LTRRescorer.java
index 27223b7..818d861 100644
--- a/solr/contrib/ltr/src/java/org/apache/solr/ltr/LTRRescorer.java
+++ b/solr/contrib/ltr/src/java/org/apache/solr/ltr/LTRRescorer.java
@@ -148,7 +148,7 @@ public class LTRRescorer extends Rescorer {
 
     LTRScoringQuery.ModelWeight.ModelScorer scorer = null;
     int hitUpto = 0;
-    final FeatureLogger<?> featureLogger = scoringQuery.getFeatureLogger();
+    final FeatureLogger featureLogger = scoringQuery.getFeatureLogger();
 
     while (hitUpto < hits.length) {
       final ScoreDoc hit = hits[hitUpto];

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c8542b2b/solr/contrib/ltr/src/java/org/apache/solr/ltr/LTRScoringQuery.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/java/org/apache/solr/ltr/LTRScoringQuery.java b/solr/contrib/ltr/src/java/org/apache/solr/ltr/LTRScoringQuery.java
index d60ebf5..b581dd5 100644
--- a/solr/contrib/ltr/src/java/org/apache/solr/ltr/LTRScoringQuery.java
+++ b/solr/contrib/ltr/src/java/org/apache/solr/ltr/LTRScoringQuery.java
@@ -63,7 +63,7 @@ public class LTRScoringQuery extends Query {
   final private Semaphore querySemaphore; // limits the number of threads per query, so that multiple requests can be serviced simultaneously
 
   // feature logger to output the features.
-  private FeatureLogger<?> fl;
+  private FeatureLogger fl;
   // Map of external parameters, such as query intent, that can be used by
   // features
   final private Map<String,String[]> efi;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c8542b2b/solr/contrib/ltr/src/java/org/apache/solr/ltr/SolrQueryRequestContextUtils.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/java/org/apache/solr/ltr/SolrQueryRequestContextUtils.java b/solr/contrib/ltr/src/java/org/apache/solr/ltr/SolrQueryRequestContextUtils.java
index 66426ea..2cff28f 100644
--- a/solr/contrib/ltr/src/java/org/apache/solr/ltr/SolrQueryRequestContextUtils.java
+++ b/solr/contrib/ltr/src/java/org/apache/solr/ltr/SolrQueryRequestContextUtils.java
@@ -37,12 +37,12 @@ public class SolrQueryRequestContextUtils {
 
   /** feature logger accessors **/
 
-  public static void setFeatureLogger(SolrQueryRequest req, FeatureLogger<?> featureLogger) {
+  public static void setFeatureLogger(SolrQueryRequest req, FeatureLogger featureLogger) {
     req.getContext().put(FEATURE_LOGGER, featureLogger);
   }
 
-  public static FeatureLogger<?> getFeatureLogger(SolrQueryRequest req) {
-    return (FeatureLogger<?>) req.getContext().get(FEATURE_LOGGER);
+  public static FeatureLogger getFeatureLogger(SolrQueryRequest req) {
+    return (FeatureLogger) req.getContext().get(FEATURE_LOGGER);
   }
 
   /** scoring query accessors **/


[08/44] lucene-solr:jira/solr-9854: SOLR-9844: Display fc total size only when field entries asked for

Posted by ab...@apache.org.
SOLR-9844: Display fc total size only when field entries asked for


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

Branch: refs/heads/jira/solr-9854
Commit: 512374384a8984c56c91f47dcac4aaf0490eda54
Parents: 7dec783
Author: Varun Thacker <va...@apache.org>
Authored: Tue Dec 13 15:52:17 2016 -0800
Committer: Varun Thacker <va...@apache.org>
Committed: Wed Dec 14 10:52:26 2016 -0800

----------------------------------------------------------------------
 .../apache/solr/search/SolrFieldCacheMBean.java |  6 ++---
 .../solr/uninverting/UninvertingReader.java     | 25 +++++++++++++-------
 .../solr/search/TestSolrFieldCacheMBean.java    |  3 ++-
 3 files changed, 22 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/51237438/solr/core/src/java/org/apache/solr/search/SolrFieldCacheMBean.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/SolrFieldCacheMBean.java b/solr/core/src/java/org/apache/solr/search/SolrFieldCacheMBean.java
index 70781e9..642b708 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrFieldCacheMBean.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrFieldCacheMBean.java
@@ -62,15 +62,15 @@ public class SolrFieldCacheMBean implements JmxAugmentedSolrInfoMBean {
   private NamedList getStats(boolean listEntries) {
     NamedList stats = new SimpleOrderedMap();
     if (listEntries) {
-      String[] entries = UninvertingReader.getUninvertedStats();
+      UninvertingReader.FieldCacheStats fieldCacheStats = UninvertingReader.getUninvertedStats();
+      String[] entries = fieldCacheStats.info;
       stats.add("entries_count", entries.length);
-      stats.add("total_size", UninvertingReader.getTotalSize());
+      stats.add("total_size", fieldCacheStats.totalSize);
       for (int i = 0; i < entries.length; i++) {
         stats.add("entry#" + i, entries[i]);
       }
     } else {
       stats.add("entries_count", UninvertingReader.getUninvertedStatsSize());
-      stats.add("total_size", UninvertingReader.getTotalSize());
     }
     return stats;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/51237438/solr/core/src/java/org/apache/solr/uninverting/UninvertingReader.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/uninverting/UninvertingReader.java b/solr/core/src/java/org/apache/solr/uninverting/UninvertingReader.java
index 87fb7a6..5276ca9 100644
--- a/solr/core/src/java/org/apache/solr/uninverting/UninvertingReader.java
+++ b/solr/core/src/java/org/apache/solr/uninverting/UninvertingReader.java
@@ -375,25 +375,34 @@ public class UninvertingReader extends FilterLeafReader {
    * Return information about the backing cache
    * @lucene.internal 
    */
-  public static String[] getUninvertedStats() {
+  public static FieldCacheStats getUninvertedStats() {
     CacheEntry[] entries = FieldCache.DEFAULT.getCacheEntries();
+    long totalBytesUsed = 0;
     String[] info = new String[entries.length];
     for (int i = 0; i < entries.length; i++) {
       info[i] = entries[i].toString();
+      totalBytesUsed += entries[i].getValue().ramBytesUsed();
     }
-    return info;
+    String totalSize = RamUsageEstimator.humanReadableUnits(totalBytesUsed);
+    return new FieldCacheStats(totalSize, info);
   }
 
   public static int getUninvertedStatsSize() {
     return FieldCache.DEFAULT.getCacheEntries().length;
   }
 
-  public static String getTotalSize() {
-    CacheEntry[] entries = FieldCache.DEFAULT.getCacheEntries();
-    long totalBytesUsed = 0;
-    for (int i = 0; i < entries.length; i++) {
-      totalBytesUsed += entries[i].getValue().ramBytesUsed();
+  /**
+   * Return information about the backing cache
+   * @lucene.internal
+   */
+  public static class FieldCacheStats {
+    public String totalSize;
+    public String[] info;
+
+    public FieldCacheStats(String totalSize, String[] info) {
+      this.totalSize = totalSize;
+      this.info = info;
     }
-    return RamUsageEstimator.humanReadableUnits(totalBytesUsed);
+
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/51237438/solr/core/src/test/org/apache/solr/search/TestSolrFieldCacheMBean.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestSolrFieldCacheMBean.java b/solr/core/src/test/org/apache/solr/search/TestSolrFieldCacheMBean.java
index a705e1e..35bdec6 100644
--- a/solr/core/src/test/org/apache/solr/search/TestSolrFieldCacheMBean.java
+++ b/solr/core/src/test/org/apache/solr/search/TestSolrFieldCacheMBean.java
@@ -69,6 +69,7 @@ public class TestSolrFieldCacheMBean extends SolrTestCaseJ4 {
     SolrFieldCacheMBean mbean = new SolrFieldCacheMBean();
     NamedList stats = checkJmx ? mbean.getStatisticsForJmx() : mbean.getStatistics();
     assert(new Integer(stats.get("entries_count").toString()) > 0);
+    assertNotNull(stats.get("total_size"));
     assertNotNull(stats.get("entry#0"));
   }
 
@@ -76,7 +77,7 @@ public class TestSolrFieldCacheMBean extends SolrTestCaseJ4 {
     SolrFieldCacheMBean mbean = new SolrFieldCacheMBean();
     NamedList stats = checkJmx ? mbean.getStatisticsForJmx() : mbean.getStatistics();
     assert(new Integer(stats.get("entries_count").toString()) > 0);
-    assertNotNull(stats.get("total_size"));
+    assertNull(stats.get("total_size"));
     assertNull(stats.get("entry#0"));
   }
 }


[14/44] lucene-solr:jira/solr-9854: LUCENE-7572: Cache the hash code of doc values queries.

Posted by ab...@apache.org.
LUCENE-7572: Cache the hash code of doc values queries.


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

Branch: refs/heads/jira/solr-9854
Commit: ea1569e2914f9ba914b582a0801d6cb83a29529b
Parents: 268d4ac
Author: Adrien Grand <jp...@gmail.com>
Authored: Thu Dec 15 16:30:15 2016 +0100
Committer: Adrien Grand <jp...@gmail.com>
Committed: Thu Dec 15 17:17:54 2016 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   2 +
 .../apache/lucene/index/PrefixCodedTerms.java   |   4 +-
 .../lucene/search/DocValuesNumbersQuery.java    |  26 ++--
 .../lucene/search/DocValuesTermsQuery.java      |  49 ++++--
 .../org/apache/lucene/search/LongHashSet.java   | 156 +++++++++++++++++++
 .../apache/lucene/search/LongHashSetTests.java  | 100 ++++++++++++
 .../lucene/search/TestDocValuesTermsQuery.java  |   1 +
 7 files changed, 310 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ea1569e2/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 0e327d2..bacc270 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -158,6 +158,8 @@ Optimizations
   writing to disk, giving a small speedup in points-heavy use cases.
   (Mike McCandless)
 
+* LUCENE-7572: Doc values queries now cache their hash code. (Adrien Grand)
+
 Other
 
 * LUCENE-7546: Fixed references to benchmark wikipedia data and the Jenkins line-docs file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ea1569e2/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java b/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java
index 3dca3db..df1653b 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java
@@ -28,7 +28,9 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 
 /**
- * Prefix codes term instances (prefixes are shared)
+ * Prefix codes term instances (prefixes are shared). This is expected to be
+ * faster to build than a FST and might also be more compact if there are no
+ * common suffixes.
  * @lucene.internal
  */
 public class PrefixCodedTerms implements Accountable {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ea1569e2/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesNumbersQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesNumbersQuery.java b/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesNumbersQuery.java
index 0fd2244..7725703 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesNumbersQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesNumbersQuery.java
@@ -18,6 +18,7 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.HashSet;
 import java.util.Objects;
 import java.util.Set;
@@ -45,11 +46,16 @@ import org.apache.lucene.index.SortedNumericDocValues;
 public class DocValuesNumbersQuery extends Query {
 
   private final String field;
-  private final Set<Long> numbers;
+  private final LongHashSet numbers;
 
-  public DocValuesNumbersQuery(String field, Set<Long> numbers) {
+  public DocValuesNumbersQuery(String field, long[] numbers) {
     this.field = Objects.requireNonNull(field);
-    this.numbers = Objects.requireNonNull(numbers, "Set of numbers must not be null");
+    this.numbers = new LongHashSet(numbers);
+  }
+
+  public DocValuesNumbersQuery(String field, Collection<Long> numbers) {
+    this.field = Objects.requireNonNull(field);
+    this.numbers = new LongHashSet(numbers.stream().mapToLong(Long::longValue).toArray());
   }
 
   public DocValuesNumbersQuery(String field, Long... numbers) {
@@ -82,15 +88,11 @@ public class DocValuesNumbersQuery extends Query {
 
   @Override
   public String toString(String defaultField) {
-    StringBuilder sb = new StringBuilder();
-    sb.append(field).append(": [");
-    for (Long number : numbers) {
-      sb.append(number).append(", ");
-    }
-    if (numbers.size() > 0) {
-      sb.setLength(sb.length() - 2);
-    }
-    return sb.append(']').toString();
+    return new StringBuilder()
+        .append(field)
+        .append(": ")
+        .append(numbers.toString())
+        .toString();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ea1569e2/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesTermsQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesTermsQuery.java b/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesTermsQuery.java
index 6d852a8..6e30bae 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesTermsQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesTermsQuery.java
@@ -25,7 +25,10 @@ import java.util.Objects;
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.PrefixCodedTerms;
 import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.PrefixCodedTerms.TermIterator;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FixedBitSet;
@@ -91,13 +94,24 @@ import org.apache.lucene.util.LongBitSet;
 public class DocValuesTermsQuery extends Query {
 
   private final String field;
-  private final BytesRef[] terms;
+  private final PrefixCodedTerms termData;
+  private final int termDataHashCode; // cached hashcode of termData
 
   public DocValuesTermsQuery(String field, Collection<BytesRef> terms) {
     this.field = Objects.requireNonNull(field);
     Objects.requireNonNull(terms, "Collection of terms must not be null");
-    this.terms = terms.toArray(new BytesRef[terms.size()]);
-    ArrayUtil.timSort(this.terms);
+    BytesRef[] sortedTerms = terms.toArray(new BytesRef[terms.size()]);
+    ArrayUtil.timSort(sortedTerms);
+    PrefixCodedTerms.Builder builder = new PrefixCodedTerms.Builder();
+    BytesRef previous = null;
+    for (BytesRef term : sortedTerms) {
+      if (term.equals(previous) == false) {
+        builder.add(field, term);
+      }
+      previous = term;
+    }
+    termData = builder.finish();
+    termDataHashCode = termData.hashCode();
   }
 
   public DocValuesTermsQuery(String field, BytesRef... terms) {
@@ -124,26 +138,30 @@ public class DocValuesTermsQuery extends Query {
   }
 
   private boolean equalsTo(DocValuesTermsQuery other) {
-    return field.equals(other.field) &&
-           Arrays.equals(terms, other.terms);
+    // termData might be heavy to compare so check the hash code first
+    return termDataHashCode == other.termDataHashCode && 
+           termData.equals(other.termData);
   }
 
   @Override
   public int hashCode() {
-    return 31 * classHash() + Objects.hash(field, Arrays.asList(terms));
+    return 31 * classHash() + termDataHashCode;
   }
 
   @Override
   public String toString(String defaultField) {
-    StringBuilder sb = new StringBuilder();
-    sb.append(field).append(": [");
-    for (BytesRef term : terms) {
-      sb.append(term).append(", ");
-    }
-    if (terms.length > 0) {
-      sb.setLength(sb.length() - 2);
+    StringBuilder builder = new StringBuilder();
+    boolean first = true;
+    TermIterator iterator = termData.iterator();
+    for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
+      if (!first) {
+        builder.append(' ');
+      }
+      first = false;
+      builder.append(new Term(iterator.field(), term).toString());
     }
-    return sb.append(']').toString();
+
+    return builder.toString();
   }
 
   @Override
@@ -155,7 +173,8 @@ public class DocValuesTermsQuery extends Query {
         final SortedSetDocValues values = DocValues.getSortedSet(context.reader(), field);
         final LongBitSet bits = new LongBitSet(values.getValueCount());
         boolean matchesAtLeastOneTerm = false;
-        for (BytesRef term : terms) {
+        TermIterator iterator = termData.iterator();
+        for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
           final long ord = values.lookupTerm(term);
           if (ord >= 0) {
             matchesAtLeastOneTerm = true;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ea1569e2/lucene/sandbox/src/java/org/apache/lucene/search/LongHashSet.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/LongHashSet.java b/lucene/sandbox/src/java/org/apache/lucene/search/LongHashSet.java
new file mode 100644
index 0000000..3a6af5f
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/LongHashSet.java
@@ -0,0 +1,156 @@
+/*
+ * 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.lucene.search;
+
+import java.util.AbstractSet;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+import org.apache.lucene.util.packed.PackedInts;
+
+final class LongHashSet extends AbstractSet<Long> {
+
+  private static final long MISSING = Long.MIN_VALUE;
+
+  final long[] table;
+  final int mask;
+  final boolean hasMissingValue;
+  final int size;
+  final int hashCode;
+
+  LongHashSet(long... values) {
+    int tableSize = Math.toIntExact(values.length * 3L / 2);
+    tableSize = 1 << PackedInts.bitsRequired(tableSize); // make it a power of 2
+    assert tableSize >= values.length * 3L / 2;
+    table = new long[tableSize];
+    Arrays.fill(table, MISSING);
+    mask = tableSize - 1;
+    boolean hasMissingValue = false;
+    int size = 0;
+    int hashCode = 0;
+    for (long value : values) {
+      if (value == MISSING || add(value)) {
+        if (value == MISSING) {
+          hasMissingValue = true;
+        }
+        ++size;
+        hashCode += Long.hashCode(value);
+      }
+    }
+    this.hasMissingValue = hasMissingValue;
+    this.size = size;
+    this.hashCode = hashCode;
+  }
+
+  private boolean add(long l) {
+    assert l != MISSING;
+    final int slot = Long.hashCode(l) & mask;
+    for (int i = slot; ; i = (i + 1) & mask) {
+      if (table[i] == MISSING) {
+        table[i] = l;
+        return true;
+      } else if (table[i] == l) {
+        // already added
+        return false;
+      }
+    }
+  }
+
+  boolean contains(long l) {
+    if (l == MISSING) {
+      return hasMissingValue;
+    }
+    final int slot = Long.hashCode(l) & mask;
+    for (int i = slot; ; i = (i + 1) & mask) {
+      if (table[i] == MISSING) {
+        return false;
+      } else if (table[i] == l) {
+        return true;
+      }
+    }
+  }
+
+  @Override
+  public int size() {
+    return size;
+  }
+
+  @Override
+  public int hashCode() {
+    return hashCode;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj != null && obj.getClass() == LongHashSet.class) {
+      LongHashSet that = (LongHashSet) obj;
+      if (hashCode != that.hashCode
+          || size != that.size
+          || hasMissingValue != that.hasMissingValue) {
+        return false;
+      }
+      for (long v : table) {
+        if (v != MISSING && that.contains(v) == false) {
+          return false;
+        }
+      }
+      return true;
+    }
+    return super.equals(obj);
+  }
+
+  @Override
+  public boolean contains(Object o) {
+    return o instanceof Long && contains(((Long) o).longValue());
+  }
+
+  @Override
+  public Iterator<Long> iterator() {
+    return new Iterator<Long>() {
+
+      private boolean hasNext = hasMissingValue;
+      private int i = -1;
+      private long value = MISSING;
+
+      @Override
+      public boolean hasNext() {
+        if (hasNext) {
+          return true;
+        }
+        while (++i < table.length) {
+          value = table[i];
+          if (value != MISSING) {
+            return hasNext = true;
+          }
+        }
+        return false;
+      }
+
+      @Override
+      public Long next() {
+        if (hasNext() == false) {
+          throw new NoSuchElementException();
+        }
+        hasNext = false;
+        return value;
+      }
+
+    };
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ea1569e2/lucene/sandbox/src/test/org/apache/lucene/search/LongHashSetTests.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/LongHashSetTests.java b/lucene/sandbox/src/test/org/apache/lucene/search/LongHashSetTests.java
new file mode 100644
index 0000000..25d94a6
--- /dev/null
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/LongHashSetTests.java
@@ -0,0 +1,100 @@
+/*
+ * 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.lucene.search;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.LongStream;
+
+import org.apache.lucene.util.LuceneTestCase;
+
+public class LongHashSetTests extends LuceneTestCase {
+
+  private void assertEquals(Set<Long> set1, LongHashSet set2) {
+    LuceneTestCase.assertEquals(set1, set2);
+    LuceneTestCase.assertEquals(set2, set1);
+    LuceneTestCase.assertEquals(set2, set2);
+    assertEquals(set1.hashCode(), set2.hashCode());
+
+    if (set1.isEmpty() == false) {
+      Set<Long> set3 = new HashSet<>(set1);
+      long removed = set3.iterator().next();
+      while (true) {
+        long next = random().nextLong();
+        if (next != removed && set3.add(next)) {
+          break;
+        }
+      }
+      assertNotEquals(set3, set2);
+    }
+  }
+
+  private void assertNotEquals(Set<Long> set1, LongHashSet set2) {
+    assertFalse(set1.equals(set2));
+    assertFalse(set2.equals(set1));
+    LongHashSet set3 = new LongHashSet(set1.stream().mapToLong(Long::longValue).toArray());
+    assertFalse(set2.equals(set3));
+  }
+
+  public void testEmpty() {
+    Set<Long> set1 = new HashSet<>();
+    LongHashSet set2 = new LongHashSet();
+    assertEquals(set1, set2);
+  }
+
+  public void testOneValue() {
+    Set<Long> set1 = new HashSet<>(Arrays.asList(42L));
+    LongHashSet set2 = new LongHashSet(42);
+    assertEquals(set1, set2);
+
+    set1 = new HashSet<>(Arrays.asList(Long.MIN_VALUE));
+    set2 = new LongHashSet(Long.MIN_VALUE);
+    assertEquals(set1, set2);
+  }
+
+  public void testTwoValues() {
+    Set<Long> set1 = new HashSet<>(Arrays.asList(42L, Long.MAX_VALUE));
+    LongHashSet set2 = new LongHashSet(42, Long.MAX_VALUE);
+    assertEquals(set1, set2);
+
+    set1 = new HashSet<>(Arrays.asList(Long.MIN_VALUE, 42L));
+    set2 = new LongHashSet(Long.MIN_VALUE, 42L);
+    assertEquals(set1, set2);
+  }
+
+  public void testRandom() {
+    final int iters = atLeast(10);
+    for (int iter = 0; iter < iters; ++iter) {
+      long[] values = new long[random().nextInt(1 << random().nextInt(16))];
+      for (int i = 0; i < values.length; ++i) {
+        if (i == 0 || random().nextInt(10) < 9) {
+          values[i] = random().nextLong();
+        } else {
+          values[i] = values[random().nextInt(i)];
+        }
+      }
+      if (values.length > 0 && random().nextBoolean()) {
+        values[values.length/2] = Long.MIN_VALUE;
+      }
+      Set<Long> set1 = LongStream.of(values).mapToObj(Long::valueOf).collect(Collectors.toCollection(HashSet::new));
+      LongHashSet set2 = new LongHashSet(values);
+      assertEquals(set1, set2);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ea1569e2/lucene/sandbox/src/test/org/apache/lucene/search/TestDocValuesTermsQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/TestDocValuesTermsQuery.java b/lucene/sandbox/src/test/org/apache/lucene/search/TestDocValuesTermsQuery.java
index 6e99492..187f172 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/search/TestDocValuesTermsQuery.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/TestDocValuesTermsQuery.java
@@ -38,6 +38,7 @@ public class TestDocValuesTermsQuery extends LuceneTestCase {
 
   public void testEquals() {
     assertEquals(new DocValuesTermsQuery("foo", "bar"), new DocValuesTermsQuery("foo", "bar"));
+    assertEquals(new DocValuesTermsQuery("foo", "bar"), new DocValuesTermsQuery("foo", "bar", "bar"));
     assertEquals(new DocValuesTermsQuery("foo", "bar", "baz"), new DocValuesTermsQuery("foo", "baz", "bar"));
     assertFalse(new DocValuesTermsQuery("foo", "bar").equals(new DocValuesTermsQuery("foo2", "bar")));
     assertFalse(new DocValuesTermsQuery("foo", "bar").equals(new DocValuesTermsQuery("foo", "baz")));


[15/44] lucene-solr:jira/solr-9854: LUCENE-7587: add helper FacetQuery and MultiFacetQuery classes to simplify drill down implementation

Posted by ab...@apache.org.
LUCENE-7587: add helper FacetQuery and MultiFacetQuery classes to simplify drill down implementation


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

Branch: refs/heads/jira/solr-9854
Commit: 835296f20a17c12c66b4f043074c94e3ddd5c2b5
Parents: 3b182aa
Author: Mike McCandless <mi...@apache.org>
Authored: Fri Dec 16 09:56:51 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Fri Dec 16 09:56:51 2016 -0500

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  4 +
 .../org/apache/lucene/facet/FacetQuery.java     | 52 +++++++++++
 .../apache/lucene/facet/MultiFacetQuery.java    | 61 ++++++++++++
 .../org/apache/lucene/facet/TestFacetQuery.java | 98 ++++++++++++++++++++
 4 files changed, 215 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/835296f2/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 7e61469..47cd6e8 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -74,6 +74,10 @@ New features
 * LUCENE-7590: Added DocValuesStatsCollector to compute statistics on DocValues
   fields. (Shai Erera)
 
+* LUCENE-7587: The new FacetQuery and MultiFacetQuery helper classes
+  make it simpler to execute drill down when drill sideways counts are
+  not needed (Emmanuel Keller via Mike McCandless)
+
 Bug Fixes
 
 * LUCENE-7547: JapaneseTokenizerFactory was failing to close the

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/835296f2/lucene/facet/src/java/org/apache/lucene/facet/FacetQuery.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/FacetQuery.java b/lucene/facet/src/java/org/apache/lucene/facet/FacetQuery.java
new file mode 100644
index 0000000..ec20292
--- /dev/null
+++ b/lucene/facet/src/java/org/apache/lucene/facet/FacetQuery.java
@@ -0,0 +1,52 @@
+/*
+ * 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.lucene.facet;
+
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermQuery;
+
+/**
+ * A term {@link Query} over a {@link FacetField}.
+ * <p>
+ * <b>NOTE:</b>This helper class is an alternative to {@link DrillDownQuery}
+ * especially in cases where you don't intend to use {@link DrillSideways}
+ *
+ * @lucene.experimental
+ */
+public class FacetQuery extends TermQuery {
+
+  /**
+   * Creates a new {@code FacetQuery} filtering the query on the given dimension.
+   */
+  public FacetQuery(final FacetsConfig facetsConfig, final String dimension, final String... path) {
+    super(toTerm(facetsConfig.getDimConfig(dimension), dimension, path));
+  }
+
+  /**
+   * Creates a new {@code FacetQuery} filtering the query on the given dimension.
+   * <p>
+   * <b>NOTE:</b>Uses FacetsConfig.DEFAULT_DIM_CONFIG.
+   */
+  public FacetQuery(final String dimension, final String... path) {
+    super(toTerm(FacetsConfig.DEFAULT_DIM_CONFIG, dimension, path));
+  }
+
+  static Term toTerm(final FacetsConfig.DimConfig dimConfig, final String dimension, final String... path) {
+    return new Term(dimConfig.indexFieldName, FacetsConfig.pathToString(dimension, path));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/835296f2/lucene/facet/src/java/org/apache/lucene/facet/MultiFacetQuery.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/MultiFacetQuery.java b/lucene/facet/src/java/org/apache/lucene/facet/MultiFacetQuery.java
new file mode 100644
index 0000000..dd212c6
--- /dev/null
+++ b/lucene/facet/src/java/org/apache/lucene/facet/MultiFacetQuery.java
@@ -0,0 +1,61 @@
+/*
+ * 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.lucene.facet;
+
+import org.apache.lucene.index.Term;
+import org.apache.lucene.queries.TermsQuery;
+import org.apache.lucene.search.Query;
+
+import java.util.ArrayList;
+import java.util.Collection;
+
+/**
+ * A multi-terms {@link Query} over a {@link FacetField}.
+ * <p>
+ * <b>NOTE:</b>This helper class is an alternative to {@link DrillDownQuery}
+ * especially in cases where you don't intend to use {@link DrillSideways}
+ *
+ * @lucene.experimental
+ * @see org.apache.lucene.queries.TermsQuery
+ */
+public class MultiFacetQuery extends TermsQuery {
+
+  /**
+   * Creates a new {@code MultiFacetQuery} filtering the query on the given dimension.
+   */
+  public MultiFacetQuery(final FacetsConfig facetsConfig, final String dimension, final String[]... paths) {
+    super(toTerms(facetsConfig.getDimConfig(dimension), dimension, paths));
+  }
+
+  /**
+   * Creates a new {@code MultiFacetQuery} filtering the query on the given dimension.
+   * <p>
+   * <b>NOTE:</b>Uses FacetsConfig.DEFAULT_DIM_CONFIG.
+   */
+  public MultiFacetQuery(final String dimension, final String[]... paths) {
+    super(toTerms(FacetsConfig.DEFAULT_DIM_CONFIG, dimension, paths));
+  }
+
+  static Collection<Term> toTerms(final FacetsConfig.DimConfig dimConfig, final String dimension,
+          final String[]... paths) {
+    final Collection<Term> terms = new ArrayList<>(paths.length);
+    for (String[] path : paths)
+      terms.add(FacetQuery.toTerm(dimConfig, dimension, path));
+    return terms;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/835296f2/lucene/facet/src/test/org/apache/lucene/facet/TestFacetQuery.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/TestFacetQuery.java b/lucene/facet/src/test/org/apache/lucene/facet/TestFacetQuery.java
new file mode 100644
index 0000000..f3aa079
--- /dev/null
+++ b/lucene/facet/src/test/org/apache/lucene/facet/TestFacetQuery.java
@@ -0,0 +1,98 @@
+/*
+ * 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.lucene.facet;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.analysis.MockTokenizer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetField;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.IOUtils;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+
+public class TestFacetQuery extends FacetTestCase {
+
+  private static Directory indexDirectory;
+  private static RandomIndexWriter indexWriter;
+  private static IndexReader indexReader;
+  private static IndexSearcher searcher;
+  private static FacetsConfig config;
+
+  private static final IndexableField[] DOC_SINGLEVALUED =
+          new IndexableField[] { new SortedSetDocValuesFacetField("Author", "Mark Twain") };
+
+  private static final IndexableField[] DOC_MULTIVALUED =
+          new SortedSetDocValuesFacetField[] { new SortedSetDocValuesFacetField("Author", "Kurt Vonnegut") };
+
+  private static final IndexableField[] DOC_NOFACET =
+          new IndexableField[] { new TextField("Hello", "World", Field.Store.YES) };
+
+  @BeforeClass
+  public static void createTestIndex() throws IOException {
+    indexDirectory = newDirectory();
+    // create and open an index writer
+    indexWriter = new RandomIndexWriter(random(), indexDirectory,
+            newIndexWriterConfig(new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false)));
+
+    config = new FacetsConfig();
+
+    indexDocuments(DOC_SINGLEVALUED, DOC_MULTIVALUED, DOC_NOFACET);
+
+    indexReader = indexWriter.getReader();
+    // prepare searcher to search against
+    searcher = newSearcher(indexReader);
+  }
+
+  private static void indexDocuments(IndexableField[]... docs) throws IOException {
+    for (IndexableField[] fields : docs) {
+      for (IndexableField field : fields) {
+        Document doc = new Document();
+        doc.add(field);
+        indexWriter.addDocument(config.build(doc));
+      }
+    }
+  }
+
+  @AfterClass
+  public static void closeTestIndex() throws IOException {
+    IOUtils.close(indexReader, indexWriter, indexDirectory);
+  }
+
+  @Test
+  public void testSingleValued() throws Exception {
+    TopDocs topDocs = searcher.search(new FacetQuery("Author", "Mark Twain"), 10);
+    assertEquals(1, topDocs.totalHits);
+  }
+
+  @Test
+  public void testMultiValued() throws Exception {
+    TopDocs topDocs = searcher.search(
+            new MultiFacetQuery("Author", new String[] { "Mark Twain" }, new String[] { "Kurt Vonnegut" }), 10);
+    assertEquals(2, topDocs.totalHits);
+  }
+}


[42/44] lucene-solr:jira/solr-9854: Squashed commit of branch 'feature/metrics', containing: SOLR-4735: Improve Solr metrics reporting SOLR-9812: Implement /admin/metrics API SOLR-9805: Use metrics-jvm library to instrument jvm internals

Posted by ab...@apache.org.
Squashed commit of branch 'feature/metrics', containing:
    SOLR-4735: Improve Solr metrics reporting
    SOLR-9812: Implement /admin/metrics API
    SOLR-9805: Use metrics-jvm library to instrument jvm internals
    SOLR-9788: Use instrumented jetty classes


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

Branch: refs/heads/jira/solr-9854
Commit: 8bbdb6248c5de3f3bd61501ba42a50aeec29c78b
Parents: 84bbb8f
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Tue Dec 20 09:31:24 2016 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Tue Dec 20 09:31:24 2016 +0100

----------------------------------------------------------------------
 .../idea/solr/contrib/analytics/analytics.iml   |   1 +
 lucene/ivy-versions.properties                  |   2 +
 solr/CHANGES.txt                                |  13 +
 .../plugin/AnalyticsStatisticsCollector.java    |   4 +-
 solr/core/ivy.xml                               |   1 -
 .../apache/solr/cloud/OverseerStatusCmd.java    |   4 +-
 .../org/apache/solr/core/CoreContainer.java     |  66 +-
 .../java/org/apache/solr/core/NodeConfig.java   |  18 +-
 .../java/org/apache/solr/core/PluginInfo.java   |   4 +-
 .../src/java/org/apache/solr/core/SolrCore.java |  73 +++
 .../org/apache/solr/core/SolrInfoMBean.java     |  10 +-
 .../org/apache/solr/core/SolrXmlConfig.java     |  14 +-
 .../apache/solr/handler/RequestHandlerBase.java |  58 +-
 .../solr/handler/admin/MetricsHandler.java      | 164 +++++
 .../solr/metrics/SolrCoreMetricManager.java     | 154 +++++
 .../org/apache/solr/metrics/SolrMetricInfo.java | 104 +++
 .../apache/solr/metrics/SolrMetricManager.java  | 652 +++++++++++++++++++
 .../apache/solr/metrics/SolrMetricProducer.java |  38 ++
 .../apache/solr/metrics/SolrMetricReporter.java |  83 +++
 .../org/apache/solr/metrics/package-info.java   |  23 +
 .../solr/metrics/reporters/SolrJmxReporter.java | 284 ++++++++
 .../solr/metrics/reporters/package-info.java    |  22 +
 .../solr/security/PermissionNameProvider.java   |   1 +
 .../apache/solr/servlet/SolrDispatchFilter.java |  27 +
 .../src/java/org/apache/solr/util/JmxUtil.java  |  78 +++
 .../org/apache/solr/util/stats/MetricUtils.java | 144 ++++
 .../org/apache/solr/util/stats/TimerUtils.java  |  58 --
 .../src/test-files/solr/solr-metricreporter.xml |  57 ++
 .../solr/handler/admin/MetricsHandlerTest.java  |  97 +++
 .../solr/metrics/SolrCoreMetricManagerTest.java | 172 +++++
 .../solr/metrics/SolrMetricManagerTest.java     | 273 ++++++++
 .../solr/metrics/SolrMetricReporterTest.java    |  69 ++
 .../solr/metrics/SolrMetricTestUtils.java       | 140 ++++
 .../metrics/SolrMetricsIntegrationTest.java     | 143 ++++
 .../metrics/reporters/MockMetricReporter.java   |  80 +++
 .../metrics/reporters/SolrJmxReporterTest.java  | 156 +++++
 .../apache/solr/util/stats/MetricUtilsTest.java |  58 ++
 .../apache/solr/util/stats/TimerUtilsTest.java  |  58 --
 solr/licenses/metrics-jetty-LICENSE-ASL.txt     | 203 ++++++
 solr/licenses/metrics-jetty-NOTICE.txt          |  12 +
 solr/licenses/metrics-jetty9-3.1.2.jar.sha1     |   1 +
 solr/licenses/metrics-json-LICENSE-ASL.txt      | 203 ++++++
 solr/licenses/metrics-json-NOTICE.txt           |  12 +
 solr/licenses/metrics-jvm-3.1.2.jar.sha1        |   1 +
 solr/licenses/metrics-jvm-LICENSE-ASL.txt       | 203 ++++++
 solr/licenses/metrics-jvm-NOTICE.txt            |  12 +
 solr/licenses/metrics-servlets-LICENSE-ASL.txt  | 203 ++++++
 solr/licenses/metrics-servlets-NOTICE.txt       |  12 +
 solr/server/build.xml                           |   4 +-
 solr/server/etc/jetty.xml                       |  20 +-
 solr/server/ivy.xml                             |  11 +-
 .../apache/solr/common/params/CommonParams.java |   4 +-
 .../java/org/apache/solr/util/TestHarness.java  |   9 +
 53 files changed, 4145 insertions(+), 168 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/dev-tools/idea/solr/contrib/analytics/analytics.iml
----------------------------------------------------------------------
diff --git a/dev-tools/idea/solr/contrib/analytics/analytics.iml b/dev-tools/idea/solr/contrib/analytics/analytics.iml
index 10f51a7..d63d3e2 100644
--- a/dev-tools/idea/solr/contrib/analytics/analytics.iml
+++ b/dev-tools/idea/solr/contrib/analytics/analytics.iml
@@ -14,6 +14,7 @@
     <orderEntry type="library" scope="TEST" name="JUnit" level="project" />
     <orderEntry type="library" name="Solr core library" level="project" />
     <orderEntry type="library" name="Solrj library" level="project" />
+    <orderEntry type="library" name="Solr example library" level="project" />
     <orderEntry type="module" scope="TEST" module-name="lucene-test-framework" />
     <orderEntry type="module" scope="TEST" module-name="solr-test-framework" />
     <orderEntry type="module" module-name="lucene-core" />

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/lucene/ivy-versions.properties
----------------------------------------------------------------------
diff --git a/lucene/ivy-versions.properties b/lucene/ivy-versions.properties
index ffc54a8..2f44f7e 100644
--- a/lucene/ivy-versions.properties
+++ b/lucene/ivy-versions.properties
@@ -76,6 +76,8 @@ com.sun.jersey.version = 1.9
 io.dropwizard.metrics.version = 3.1.2
 /io.dropwizard.metrics/metrics-core = ${io.dropwizard.metrics.version}
 /io.dropwizard.metrics/metrics-healthchecks = ${io.dropwizard.metrics.version}
+/io.dropwizard.metrics/metrics-jetty9 = ${io.dropwizard.metrics.version}
+/io.dropwizard.metrics/metrics-jvm = ${io.dropwizard.metrics.version}
 
 io.netty.netty-all.version = 4.0.36.Final
 /io.netty/netty-all = ${io.netty.netty-all.version}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index d04d491..519cdfa 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -173,11 +173,24 @@ New Features
 * SOLR-9844: FieldCache information fetched via the mbeans handler or seen via the UI now displays the total size used.
   The individual cache entries in the response are now formatted better as well. (Varun Thacker)
 
+<<<<<<< HEAD
 * SOLR-9513: Generic authentication plugins (GenericHadoopAuthPlugin and ConfigurableInternodeAuthHadoopPlugin) that delegate
   all functionality to Hadoop authentication framework. (Hrishikesh Gadre via Ishan Chattopadhyaya)
 
 * SOLR-9860: Enable configuring invariantParams via HttpSolrClient.Builder (Hrishikesh Gadre, Ishan Chattopadhyaya)
  
+* SOLR-4735: Improve metrics reporting. This uses the dropwizard metric library, adding an internal API
+  for registering and reporting metrics from Solr components. Several new metrics and an improved JMX
+  reporter have been added (Alan Woodward, Jeff Wartes, Christine Poerschke, Kelvin Wong, shalin, ab)
+
+* SOLR-9788: Use instrumented jetty classes provided by the dropwizard metric library. (shalin)
+
+* SOLR-9805: Use metrics-jvm library to instrument jvm internals such as GC, memory usage and others. (shalin)
+
+* SOLR-9812: Added a new /admin/metrics API to return all metrics collected by Solr via API. API supports two
+  optional parameters 'group' (all,jvm,jetty,http,node,core) and 'type' (all,counter,timer,gauge,histogram) both
+  of which are multi-valued. Example: http://localhost:8983/solr/admin/metrics?group=jvm,jetty&type=counter
+  (shalin)
 
 Optimizations
 ----------------------

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/ivy.xml
----------------------------------------------------------------------
diff --git a/solr/core/ivy.xml b/solr/core/ivy.xml
index 5710da9..67e4379 100644
--- a/solr/core/ivy.xml
+++ b/solr/core/ivy.xml
@@ -50,7 +50,6 @@
     <dependency org="log4j" name="log4j" rev="${/log4j/log4j}" conf="compile"/>
     <dependency org="org.slf4j" name="slf4j-log4j12" rev="${/org.slf4j/slf4j-log4j12}" conf="compile"/>
     <dependency org="org.slf4j" name="jcl-over-slf4j" rev="${/org.slf4j/jcl-over-slf4j}" conf="compile"/>
-    <dependency org="io.dropwizard.metrics" name="metrics-core" rev="${/io.dropwizard.metrics/metrics-core}" conf="compile" />
 
     <dependency org="org.easymock" name="easymock" rev="${/org.easymock/easymock}" conf="test"/>
     <dependency org="cglib" name="cglib-nodep" rev="${/cglib/cglib-nodep}" conf="test"/>

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/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 7c38b81..6e640bc 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -33,6 +33,7 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
 
+import com.codahale.metrics.Gauge;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
 import org.apache.http.auth.AuthSchemeProvider;
@@ -58,6 +59,7 @@ import org.apache.solr.handler.admin.CollectionsHandler;
 import org.apache.solr.handler.admin.ConfigSetsHandler;
 import org.apache.solr.handler.admin.CoreAdminHandler;
 import org.apache.solr.handler.admin.InfoHandler;
+import org.apache.solr.handler.admin.MetricsHandler;
 import org.apache.solr.handler.admin.SecurityConfHandler;
 import org.apache.solr.handler.admin.SecurityConfHandlerLocal;
 import org.apache.solr.handler.admin.SecurityConfHandlerZk;
@@ -65,6 +67,8 @@ import org.apache.solr.handler.admin.ZookeeperInfoHandler;
 import org.apache.solr.handler.component.ShardHandlerFactory;
 import org.apache.solr.logging.LogWatcher;
 import org.apache.solr.logging.MDCLoggingContext;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.metrics.SolrMetricProducer;
 import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.security.AuthenticationPlugin;
 import org.apache.solr.security.AuthorizationPlugin;
@@ -85,6 +89,7 @@ import static org.apache.solr.common.params.CommonParams.COLLECTIONS_HANDLER_PAT
 import static org.apache.solr.common.params.CommonParams.CONFIGSETS_HANDLER_PATH;
 import static org.apache.solr.common.params.CommonParams.CORES_HANDLER_PATH;
 import static org.apache.solr.common.params.CommonParams.INFO_HANDLER_PATH;
+import static org.apache.solr.common.params.CommonParams.METRICS_PATH;
 import static org.apache.solr.common.params.CommonParams.ZK_PATH;
 import static org.apache.solr.security.AuthenticationPlugin.AUTHENTICATION_PLUGIN_PROP;
 
@@ -156,6 +161,10 @@ public class CoreContainer {
 
   private BackupRepositoryFactory backupRepoFactory;
 
+  protected SolrMetricManager metricManager;
+
+  protected MetricsHandler metricsHandler;
+
   /**
    * This method instantiates a new instance of {@linkplain BackupRepository}.
    *
@@ -423,6 +432,10 @@ public class CoreContainer {
     return pkiAuthenticationPlugin;
   }
 
+  public SolrMetricManager getMetricManager() {
+    return metricManager;
+  }
+
   //-------------------------------------------------------------------
   // Initialization / Cleanup
   //-------------------------------------------------------------------
@@ -463,28 +476,45 @@ public class CoreContainer {
 
     MDCLoggingContext.setNode(this);
 
+    metricManager = new SolrMetricManager();
+
     securityConfHandler = isZooKeeperAware() ? new SecurityConfHandlerZk(this) : new SecurityConfHandlerLocal(this);
     reloadSecurityProperties();
     this.backupRepoFactory = new BackupRepositoryFactory(cfg.getBackupRepositoryPlugins());
 
-    containerHandlers.put(ZK_PATH, new ZookeeperInfoHandler(this));
-    collectionsHandler = createHandler(cfg.getCollectionsHandlerClass(), CollectionsHandler.class);
-    containerHandlers.put(COLLECTIONS_HANDLER_PATH, collectionsHandler);
-    infoHandler        = createHandler(cfg.getInfoHandlerClass(), InfoHandler.class);
-    containerHandlers.put(INFO_HANDLER_PATH, infoHandler);
-    coreAdminHandler   = createHandler(cfg.getCoreAdminHandlerClass(), CoreAdminHandler.class);
-    containerHandlers.put(CORES_HANDLER_PATH, coreAdminHandler);
-    configSetsHandler = createHandler(cfg.getConfigSetsHandlerClass(), ConfigSetsHandler.class);
-    containerHandlers.put(CONFIGSETS_HANDLER_PATH, configSetsHandler);
+    createHandler(ZK_PATH, ZookeeperInfoHandler.class.getName(), ZookeeperInfoHandler.class);
+    collectionsHandler = createHandler(COLLECTIONS_HANDLER_PATH, cfg.getCollectionsHandlerClass(), CollectionsHandler.class);
+    infoHandler        = createHandler(INFO_HANDLER_PATH, cfg.getInfoHandlerClass(), InfoHandler.class);
+    coreAdminHandler   = createHandler(CORES_HANDLER_PATH, cfg.getCoreAdminHandlerClass(), CoreAdminHandler.class);
+    configSetsHandler = createHandler(CONFIGSETS_HANDLER_PATH, cfg.getConfigSetsHandlerClass(), ConfigSetsHandler.class);
+    metricsHandler = createHandler(METRICS_PATH, MetricsHandler.class.getName(), MetricsHandler.class);
     containerHandlers.put(AUTHZ_PATH, securityConfHandler);
+    securityConfHandler.initializeMetrics(metricManager, SolrInfoMBean.Group.node.toString(), AUTHZ_PATH);
     containerHandlers.put(AUTHC_PATH, securityConfHandler);
     if(pkiAuthenticationPlugin != null)
       containerHandlers.put(PKIAuthenticationPlugin.PATH, pkiAuthenticationPlugin.getRequestHandler());
 
+    metricManager.loadReporters(cfg.getMetricReporterPlugins(), loader, SolrInfoMBean.Group.node);
+    metricManager.loadReporters(cfg.getMetricReporterPlugins(), loader, SolrInfoMBean.Group.jvm);
+    metricManager.loadReporters(cfg.getMetricReporterPlugins(), loader, SolrInfoMBean.Group.jetty);
+    metricManager.loadReporters(cfg.getMetricReporterPlugins(), loader, SolrInfoMBean.Group.http);
+
     coreConfigService = ConfigSetService.createConfigSetService(cfg, loader, zkSys.zkController);
 
     containerProperties.putAll(cfg.getSolrProperties());
 
+    // initialize gauges for reporting the number of cores
+    Gauge<Integer> loadedCores = () -> solrCores.getCores().size();
+    Gauge<Integer> lazyCores = () -> solrCores.getCoreNames().size() - solrCores.getCores().size();
+    Gauge<Integer> unloadedCores = () -> solrCores.getAllCoreNames().size() - solrCores.getCoreNames().size();
+
+    metricManager.register(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node),
+        loadedCores, true, "loaded", "cores");
+    metricManager.register(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node),
+        lazyCores, true, "lazy", "cores");
+    metricManager.register(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node),
+        unloadedCores, true, "unloaded", "cores");
+
     // setup executor to load cores in parallel
     ExecutorService coreLoadExecutor = ExecutorUtil.newMDCAwareFixedThreadPool(
         cfg.getCoreLoadThreadCount(isZooKeeperAware()),
@@ -658,6 +688,10 @@ public class CoreContainer {
       }
     }
 
+    if (metricManager != null) {
+      metricManager.closeReporters(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node));
+    }
+
     // It should be safe to close the authorization plugin at this point.
     try {
       if(authorizationPlugin != null) {
@@ -1034,6 +1068,9 @@ public class CoreContainer {
     SolrCore core = solrCores.remove(name);
     coresLocator.delete(this, cd);
 
+    // delete metrics specific to this core
+    metricManager.removeRegistry(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.core, name));
+
     if (core == null) {
       // transient core
       SolrCore.deleteUnloadedCore(cd, deleteDataDir, deleteInstanceDir);
@@ -1167,8 +1204,15 @@ public class CoreContainer {
 
   // ---------------- CoreContainer request handlers --------------
 
-  protected <T> T createHandler(String handlerClass, Class<T> clazz) {
-    return loader.newInstance(handlerClass, clazz, null, new Class[] { CoreContainer.class }, new Object[] { this });
+  protected <T> T createHandler(String path, String handlerClass, Class<T> clazz) {
+    T handler = loader.newInstance(handlerClass, clazz, null, new Class[] { CoreContainer.class }, new Object[] { this });
+    if (handler instanceof SolrRequestHandler) {
+      containerHandlers.put(path, (SolrRequestHandler)handler);
+    }
+    if (handler instanceof SolrMetricProducer) {
+      ((SolrMetricProducer)handler).initializeMetrics(metricManager, SolrInfoMBean.Group.node.toString(), path);
+    }
+    return handler;
   }
 
   public CoreAdminHandler getMultiCoreHandler() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/java/org/apache/solr/core/NodeConfig.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/NodeConfig.java b/solr/core/src/java/org/apache/solr/core/NodeConfig.java
index ea451a9..258fd14 100644
--- a/solr/core/src/java/org/apache/solr/core/NodeConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/NodeConfig.java
@@ -60,13 +60,16 @@ public class NodeConfig {
 
   private final PluginInfo[] backupRepositoryPlugins;
 
+  private final PluginInfo[] metricReporterPlugins;
+
   private NodeConfig(String nodeName, Path coreRootDirectory, Path configSetBaseDirectory, String sharedLibDirectory,
                      PluginInfo shardHandlerFactoryConfig, UpdateShardHandlerConfig updateShardHandlerConfig,
                      String coreAdminHandlerClass, String collectionsAdminHandlerClass,
                      String infoHandlerClass, String configSetsHandlerClass,
                      LogWatcherConfig logWatcherConfig, CloudConfig cloudConfig, Integer coreLoadThreads,
                      int transientCacheSize, boolean useSchemaCache, String managementPath, SolrResourceLoader loader,
-                     Properties solrProperties, PluginInfo[] backupRepositoryPlugins) {
+                     Properties solrProperties, PluginInfo[] backupRepositoryPlugins,
+                     PluginInfo[] metricReporterPlugins) {
     this.nodeName = nodeName;
     this.coreRootDirectory = coreRootDirectory;
     this.configSetBaseDirectory = configSetBaseDirectory;
@@ -86,6 +89,7 @@ public class NodeConfig {
     this.loader = loader;
     this.solrProperties = solrProperties;
     this.backupRepositoryPlugins = backupRepositoryPlugins;
+    this.metricReporterPlugins = metricReporterPlugins;
 
     if (this.cloudConfig != null && this.getCoreLoadThreadCount(false) < 2) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
@@ -174,6 +178,10 @@ public class NodeConfig {
     return backupRepositoryPlugins;
   }
 
+  public PluginInfo[] getMetricReporterPlugins() {
+    return metricReporterPlugins;
+  }
+
   public static class NodeConfigBuilder {
 
     private Path coreRootDirectory;
@@ -193,6 +201,7 @@ public class NodeConfig {
     private String managementPath;
     private Properties solrProperties = new Properties();
     private PluginInfo[] backupRepositoryPlugins;
+    private PluginInfo[] metricReporterPlugins;
 
     private final SolrResourceLoader loader;
     private final String nodeName;
@@ -300,11 +309,16 @@ public class NodeConfig {
       return this;
     }
 
+    public NodeConfigBuilder setMetricReporterPlugins(PluginInfo[] metricReporterPlugins) {
+      this.metricReporterPlugins = metricReporterPlugins;
+      return this;
+    }
+
     public NodeConfig build() {
       return new NodeConfig(nodeName, coreRootDirectory, configSetBaseDirectory, sharedLibDirectory, shardHandlerFactoryConfig,
                             updateShardHandlerConfig, coreAdminHandlerClass, collectionsAdminHandlerClass, infoHandlerClass, configSetsHandlerClass,
                             logWatcherConfig, cloudConfig, coreLoadThreads, transientCacheSize, useSchemaCache, managementPath, loader, solrProperties,
-                            backupRepositoryPlugins);
+                            backupRepositoryPlugins, metricReporterPlugins);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/java/org/apache/solr/core/PluginInfo.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/PluginInfo.java b/solr/core/src/java/org/apache/solr/core/PluginInfo.java
index 5fdef3c..10f8b8d 100644
--- a/solr/core/src/java/org/apache/solr/core/PluginInfo.java
+++ b/solr/core/src/java/org/apache/solr/core/PluginInfo.java
@@ -111,6 +111,7 @@ public class PluginInfo implements MapSerializable {
     if (type != null) sb.append("type = " + type + ",");
     if (name != null) sb.append("name = " + name + ",");
     if (className != null) sb.append("class = " + className + ",");
+    if (attributes != null && attributes.size() > 0) sb.append("attributes = " + attributes + ",");
     if (initArgs != null && initArgs.size() > 0) sb.append("args = " + initArgs);
     sb.append("}");
     return sb.toString();
@@ -181,7 +182,8 @@ public class PluginInfo implements MapSerializable {
 
   }
   public PluginInfo copy() {
-    PluginInfo result = new PluginInfo(type, attributes, initArgs.clone(), children);
+    PluginInfo result = new PluginInfo(type, attributes,
+        initArgs != null ? initArgs.clone() : null, children);
     result.isFromSolrConfig = isFromSolrConfig;
     return result;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/java/org/apache/solr/core/SolrCore.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrCore.java b/solr/core/src/java/org/apache/solr/core/SolrCore.java
index f75e780..a8d7738 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -53,6 +53,8 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.locks.ReentrantLock;
 
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Timer;
 import com.google.common.collect.MapMaker;
 import org.apache.commons.io.FileUtils;
 import org.apache.lucene.analysis.util.ResourceLoader;
@@ -94,6 +96,9 @@ import org.apache.solr.handler.RequestHandlerBase;
 import org.apache.solr.handler.component.HighlightComponent;
 import org.apache.solr.handler.component.SearchComponent;
 import org.apache.solr.logging.MDCLoggingContext;
+import org.apache.solr.metrics.SolrCoreMetricManager;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.metrics.SolrMetricProducer;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.response.BinaryResponseWriter;
@@ -187,6 +192,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
   private final PluginBag<SearchComponent> searchComponents = new PluginBag<>(SearchComponent.class, this);
   private final PluginBag<UpdateRequestProcessorFactory> updateProcessors = new PluginBag<>(UpdateRequestProcessorFactory.class, this, true);
   private final Map<String,UpdateRequestProcessorChain> updateProcessorChains;
+  private final SolrCoreMetricManager coreMetricManager;
   private final Map<String, SolrInfoMBean> infoRegistry;
   private final IndexDeletionPolicyWrapper solrDelPolicy;
   private final SolrSnapshotMetaDataManager snapshotMgr;
@@ -200,6 +206,12 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
   private final ReentrantLock ruleExpiryLock;
   private final ReentrantLock snapshotDelLock; // A lock instance to guard against concurrent deletions.
 
+  private final Timer newSearcherTimer;
+  private final Timer newSearcherWarmupTimer;
+  private final Counter newSearcherCounter;
+  private final Counter newSearcherMaxReachedCounter;
+  private final Counter newSearcherOtherErrorsCounter;
+
   public Date getStartTimeStamp() { return startTime; }
 
   private final Map<Object, IndexFingerprint> perSegmentFingerprintCache = new MapMaker().weakKeys().makeMap();
@@ -386,9 +398,13 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
   }
 
   public void setName(String v) {
+    String oldName = this.name;
     this.name = v;
     this.logid = (v==null)?"":("["+v+"] ");
     this.coreDescriptor = new CoreDescriptor(v, this.coreDescriptor);
+    if (coreMetricManager != null) {
+      coreMetricManager.afterCoreSetName();
+    }
   }
 
   public String getLogId()
@@ -397,6 +413,15 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
   }
 
   /**
+   * Returns the {@link SolrCoreMetricManager} for this core.
+   *
+   * @return the {@link SolrCoreMetricManager} for this core
+   */
+  public SolrCoreMetricManager getCoreMetricManager() {
+    return coreMetricManager;
+  }
+
+  /**
    * Returns a Map of name vs SolrInfoMBean objects. The returned map is an instance of
    * a ConcurrentHashMap and therefore no synchronization is needed for putting, removing
    * or iterating over it.
@@ -838,6 +863,18 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
 
     checkVersionFieldExistsInSchema(schema, coreDescriptor);
 
+    // Initialize the metrics manager
+    this.coreMetricManager = initCoreMetricManager(config);
+
+    SolrMetricManager metricManager = this.coreDescriptor.getCoreContainer().getMetricManager();
+
+    // initialize searcher-related metrics
+    newSearcherCounter = metricManager.counter(coreMetricManager.getRegistryName(), "newSearcher");
+    newSearcherTimer = metricManager.timer(coreMetricManager.getRegistryName(), "newSearcherTime");
+    newSearcherWarmupTimer = metricManager.timer(coreMetricManager.getRegistryName(), "newSearcherWarmup");
+    newSearcherMaxReachedCounter = metricManager.counter(coreMetricManager.getRegistryName(), "newSearcherMaxReached");
+    newSearcherOtherErrorsCounter = metricManager.counter(coreMetricManager.getRegistryName(), "newSearcherErrors");
+
     // Initialize JMX
     this.infoRegistry = initInfoRegistry(name, config);
     infoRegistry.put("fieldCache", new SolrFieldCacheMBean());
@@ -1041,6 +1078,19 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
     setLatestSchema(schema);
   }
 
+  /**
+   * Initializes the core's {@link SolrCoreMetricManager} with a given configuration.
+   * If metric reporters are configured, they are also initialized for this core.
+   *
+   * @param config the given configuration
+   * @return an instance of {@link SolrCoreMetricManager}
+   */
+  private SolrCoreMetricManager initCoreMetricManager(SolrConfig config) {
+    SolrCoreMetricManager coreMetricManager = new SolrCoreMetricManager(this);
+    coreMetricManager.loadReporters();
+    return coreMetricManager;
+  }
+
   private Map<String,SolrInfoMBean> initInfoRegistry(String name, SolrConfig config) {
     if (config.jmxConfig.enabled) {
       return new JmxMonitoredMap<String, SolrInfoMBean>(name, String.valueOf(this.hashCode()), config.jmxConfig);
@@ -1361,6 +1411,15 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
       }
     }
 
+    try {
+      coreMetricManager.close();
+    } catch (Throwable e) {
+      SolrException.log(log, e);
+      if (e instanceof  Error) {
+        throw (Error) e;
+      }
+    }
+
     // Close the snapshots meta-data directory.
     Directory snapshotsDir = snapshotMgr.getSnapshotsDir();
     try {
@@ -1920,12 +1979,14 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
         // first: increment count to signal other threads that we are
         //        opening a new searcher.
         onDeckSearchers++;
+        newSearcherCounter.inc();
         if (onDeckSearchers < 1) {
           // should never happen... just a sanity check
           log.error(logid + "ERROR!!! onDeckSearchers is " + onDeckSearchers);
           onDeckSearchers = 1;  // reset
         } else if (onDeckSearchers > maxWarmingSearchers) {
           onDeckSearchers--;
+          newSearcherMaxReachedCounter.inc();
           try {
             searcherLock.wait();
           } catch (InterruptedException e) {
@@ -1947,6 +2008,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
     boolean success = false;
 
     openSearcherLock.lock();
+    Timer.Context timerContext = newSearcherTimer.time();
     try {
       searchHolder = openNewSearcher(updateHandlerReopens, false);
        // the searchHolder will be incremented once already (and it will eventually be assigned to _searcher when registered)
@@ -1989,6 +2051,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
         // should this go before the other event handlers or after?
         if (currSearcher != null) {
           future = searcherExecutor.submit(() -> {
+            Timer.Context warmupContext = newSearcherWarmupTimer.time();
             try {
               newSearcher.warm(currSearcher);
             } catch (Throwable e) {
@@ -1996,6 +2059,8 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
               if (e instanceof Error) {
                 throw (Error) e;
               }
+            } finally {
+              warmupContext.close();
             }
             return null;
           });
@@ -2076,7 +2141,10 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
     } finally {
 
+      timerContext.close();
+
       if (!success) {
+        newSearcherOtherErrorsCounter.inc();;
         synchronized (searcherLock) {
           onDeckSearchers--;
 
@@ -2750,6 +2818,11 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
 
   public void registerInfoBean(String name, SolrInfoMBean solrInfoMBean) {
     infoRegistry.put(name, solrInfoMBean);
+
+    if (solrInfoMBean instanceof SolrMetricProducer) {
+      SolrMetricProducer producer = (SolrMetricProducer) solrInfoMBean;
+      coreMetricManager.registerMetricProducer(name, producer);
+    }
   }
 
   private static boolean checkStale(SolrZkClient zkClient,  String zkPath, int currentVersion)  {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java b/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java
index c119e2f..c5fb84b 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java
@@ -29,7 +29,15 @@ import org.apache.solr.common.util.NamedList;
  */
 public interface SolrInfoMBean {
 
-  public enum Category { CORE, QUERYHANDLER, UPDATEHANDLER, CACHE, HIGHLIGHTING, QUERYPARSER, OTHER };
+  /**
+   * Category of {@link SolrCore} component.
+   */
+  enum Category { CORE, QUERYHANDLER, UPDATEHANDLER, CACHE, HIGHLIGHTING, QUERYPARSER, OTHER }
+
+  /**
+   * Top-level group of beans for a subsystem.
+   */
+  enum Group { jvm, jetty, http, node, core }
 
   /**
    * Simple common usage name, e.g. BasicQueryHandler,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java b/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
index 65b248d..49d9ae5 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
@@ -95,7 +95,8 @@ public class SolrXmlConfig {
     configBuilder.setSolrProperties(loadProperties(config));
     if (cloudConfig != null)
       configBuilder.setCloudConfig(cloudConfig);
-    configBuilder.setBackupRepositoryPlugins((getBackupRepositoryPluginInfos(config)));
+    configBuilder.setBackupRepositoryPlugins(getBackupRepositoryPluginInfos(config));
+    configBuilder.setMetricReporterPlugins(getMetricReporterPluginInfos(config));
     return fillSolrSection(configBuilder, entries);
   }
 
@@ -436,5 +437,16 @@ public class SolrXmlConfig {
     }
     return configs;
   }
+
+  private static PluginInfo[] getMetricReporterPluginInfos(Config config) {
+    NodeList nodes = (NodeList) config.evaluate("solr/metrics/reporter", XPathConstants.NODESET);
+    if (nodes == null || nodes.getLength() == 0)
+      return new PluginInfo[0];
+    PluginInfo[] configs = new PluginInfo[nodes.getLength()];
+    for (int i = 0; i < nodes.getLength(); i++) {
+      configs[i] = new PluginInfo(nodes.item(i), "SolrMetricReporter", true, true);
+    }
+    return configs;
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java b/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
index 218b6de..85597dc 100644
--- a/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
+++ b/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
@@ -18,8 +18,11 @@ package org.apache.solr.handler;
 
 import java.lang.invoke.MethodHandles;
 import java.net.URL;
-import java.util.concurrent.atomic.LongAdder;
+import java.util.Arrays;
+import java.util.Collection;
 
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Meter;
 import com.codahale.metrics.Timer;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.SolrParams;
@@ -29,12 +32,14 @@ import org.apache.solr.common.util.SuppressForbidden;
 import org.apache.solr.core.PluginBag;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.SolrInfoMBean;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.metrics.SolrMetricProducer;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.search.SyntaxError;
 import org.apache.solr.util.SolrPluginUtils;
-import org.apache.solr.util.stats.TimerUtils;
+import org.apache.solr.util.stats.MetricUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -43,7 +48,7 @@ import static org.apache.solr.core.RequestParams.USEPARAM;
 /**
  *
  */
-public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfoMBean, NestedRequestHandler {
+public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfoMBean, SolrMetricProducer, NestedRequestHandler {
 
   protected NamedList initArgs = null;
   protected SolrParams defaults;
@@ -52,11 +57,12 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
   protected boolean httpCaching = true;
 
   // Statistics
-  private final LongAdder numRequests = new LongAdder();
-  private final LongAdder numServerErrors = new LongAdder();
-  private final LongAdder numClientErrors = new LongAdder();
-  private final LongAdder numTimeouts = new LongAdder();
-  private final Timer requestTimes = new Timer();
+  private Meter numErrors = new Meter();
+  private Meter numServerErrors = new Meter();
+  private Meter numClientErrors = new Meter();
+  private Meter numTimeouts = new Meter();
+  private Counter requests = new Counter();
+  private Timer requestTimes = new Timer();
 
   private final long handlerStart;
 
@@ -126,6 +132,17 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
 
   }
 
+  @Override
+  public Collection<String> initializeMetrics(SolrMetricManager manager, String registryName, String scope) {
+    numErrors = manager.meter(registryName, "errors", getCategory().toString(), scope);
+    numServerErrors = manager.meter(registryName, "serverErrors", getCategory().toString(), scope);
+    numClientErrors = manager.meter(registryName, "clientErrors", getCategory().toString(), scope);
+    numTimeouts = manager.meter(registryName, "timeouts", getCategory().toString(), scope);
+    requests = manager.counter(registryName, "requests", getCategory().toString(), scope);
+    requestTimes = manager.timer(registryName, "requestTimes", getCategory().toString(), scope);
+    return Arrays.asList("errors", "serverErrors", "clientErrors", "timeouts", "requestTimes", "requests");
+  }
+
   public static SolrParams getSolrParamsFromNamedList(NamedList args, String key) {
     Object o = args.get(key);
     if (o != null && o instanceof NamedList) {
@@ -142,7 +159,7 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
 
   @Override
   public void handleRequest(SolrQueryRequest req, SolrQueryResponse rsp) {
-    numRequests.increment();
+    requests.inc();
     Timer.Context timer = requestTimes.time();
     try {
       if(pluginInfo != null && pluginInfo.attributes.containsKey(USEPARAM)) req.getContext().put(USEPARAM,pluginInfo.attributes.get(USEPARAM));
@@ -156,7 +173,7 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
         Object partialResults = header.get(SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_KEY);
         boolean timedOut = partialResults == null ? false : (Boolean)partialResults;
         if( timedOut ) {
-          numTimeouts.increment();
+          numTimeouts.mark();
           rsp.setHttpCaching(false);
         }
       }
@@ -182,14 +199,14 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
       if (incrementErrors) {
         SolrException.log(log, e);
 
+        numErrors.mark();
         if (isServerError) {
-          numServerErrors.increment();
+          numServerErrors.mark();
         } else {
-          numClientErrors.increment();
+          numClientErrors.mark();
         }
       }
-    }
-    finally {
+    } finally {
       timer.stop();
     }
   }
@@ -268,15 +285,14 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
   public NamedList<Object> getStatistics() {
     NamedList<Object> lst = new SimpleOrderedMap<>();
     lst.add("handlerStart",handlerStart);
-    lst.add("requests", numRequests.longValue());
-    lst.add("errors", numServerErrors.longValue() + numClientErrors.longValue());
-    lst.add("serverErrors", numServerErrors.longValue());
-    lst.add("clientErrors", numClientErrors.longValue());
-    lst.add("timeouts", numTimeouts.longValue());
-    TimerUtils.addMetrics(lst, requestTimes);
+    lst.add("requests", requests.getCount());
+    lst.add("errors", numErrors.getCount());
+    lst.add("serverErrors", numServerErrors.getCount());
+    lst.add("clientErrors", numClientErrors.getCount());
+    lst.add("timeouts", numTimeouts.getCount());
+    MetricUtils.addMetrics(lst, requestTimes);
     return lst;
   }
-
 }
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java
new file mode 100644
index 0000000..78b2045
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.handler.admin;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.Meter;
+import com.codahale.metrics.MetricFilter;
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.Timer;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.handler.RequestHandlerBase;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.security.AuthorizationContext;
+import org.apache.solr.security.PermissionNameProvider;
+import org.apache.solr.util.stats.MetricUtils;
+
+/**
+ * Request handler to return metrics
+ */
+public class MetricsHandler extends RequestHandlerBase implements PermissionNameProvider {
+  final CoreContainer container;
+  final SolrMetricManager metricManager;
+
+  public MetricsHandler() {
+    this.container = null;
+    this.metricManager = null;
+  }
+
+  public MetricsHandler(CoreContainer container) {
+    this.container = container;
+    this.metricManager = this.container.getMetricManager();
+  }
+
+  @Override
+  public Name getPermissionName(AuthorizationContext request) {
+    return Name.METRICS_READ_PERM;
+  }
+
+  @Override
+  public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
+    if (container == null) {
+      throw new SolrException(SolrException.ErrorCode.INVALID_STATE, "Core container instance not initialized");
+    }
+
+    List<MetricType> metricTypes = parseMetricTypes(req);
+    List<MetricFilter> metricFilters = metricTypes.stream().map(MetricType::asMetricFilter).collect(Collectors.toList());
+    List<Group> requestedGroups = parseGroups(req);
+
+    NamedList response = new NamedList();
+    for (Group group : requestedGroups) {
+      String registryName = SolrMetricManager.getRegistryName(group);
+      if (group == Group.core) {
+        // this requires special handling because of the way we create registry name for a core (deeply nested)
+        container.getAllCoreNames().forEach(s -> {
+          String coreRegistryName;
+          try (SolrCore core = container.getCore(s)) {
+            coreRegistryName = core.getCoreMetricManager().getRegistryName();
+          }
+          MetricRegistry registry = metricManager.registry(coreRegistryName);
+          response.add(coreRegistryName, MetricUtils.toNamedList(registry, metricFilters));
+        });
+      } else {
+        MetricRegistry registry = metricManager.registry(registryName);
+        response.add(registryName, MetricUtils.toNamedList(registry, metricFilters));
+      }
+    }
+    rsp.getValues().add("metrics", response);
+  }
+
+  private List<Group> parseGroups(SolrQueryRequest req) {
+    String[] groupStr = req.getParams().getParams("group");
+    List<String> groups = Collections.emptyList();
+    if (groupStr != null && groupStr.length > 0) {
+      groups = new ArrayList<>();
+      for (String g : groupStr) {
+        groups.addAll(StrUtils.splitSmart(g, ','));
+      }
+    }
+
+    List<Group> requestedGroups = Arrays.asList(Group.values()); // by default we return all groups
+    try {
+      if (groups.size() > 0 && !groups.contains("all")) {
+        requestedGroups = groups.stream().map(String::trim).map(Group::valueOf).collect(Collectors.toList());
+      }
+    } catch (IllegalArgumentException e) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Invalid group in: " + groups + " specified. Must be one of (all, jvm, jetty, http, node, core)", e);
+    }
+    return requestedGroups;
+  }
+
+  private List<MetricType> parseMetricTypes(SolrQueryRequest req) {
+    String[] typeStr = req.getParams().getParams("type");
+    List<String> types = Collections.emptyList();
+    if (typeStr != null && typeStr.length > 0)  {
+      types = new ArrayList<>();
+      for (String type : typeStr) {
+        types.addAll(StrUtils.splitSmart(type, ','));
+      }
+    }
+
+    List<MetricType> metricTypes = Collections.singletonList(MetricType.all); // include all metrics by default
+    try {
+      if (types.size() > 0) {
+        metricTypes = types.stream().map(String::trim).map(MetricType::valueOf).collect(Collectors.toList());
+      }
+    } catch (IllegalArgumentException e) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Invalid metric type in: " + types + " specified. Must be one of (all, meter, timer, histogram, counter, gauge)", e);
+    }
+    return metricTypes;
+  }
+
+  @Override
+  public String getDescription() {
+    return "A handler to return all the metrics gathered by Solr";
+  }
+
+  enum MetricType {
+    histogram(Histogram.class),
+    meter(Meter.class),
+    timer(Timer.class),
+    counter(Counter.class),
+    gauge(Gauge.class),
+    all(null);
+
+    private final Class klass;
+
+    MetricType(Class klass) {
+      this.klass = klass;
+    }
+
+    public MetricFilter asMetricFilter() {
+      return (name, metric) -> klass == null || klass.isInstance(metric);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java b/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java
new file mode 100644
index 0000000..0e5403b
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java
@@ -0,0 +1,154 @@
+/*
+ * 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;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collection;
+import java.lang.invoke.MethodHandles;
+
+import org.apache.solr.core.NodeConfig;
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.SolrInfoMBean;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Responsible for collecting metrics from {@link SolrMetricProducer}'s
+ * and exposing metrics to {@link SolrMetricReporter}'s.
+ */
+public class SolrCoreMetricManager implements Closeable {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private final SolrCore core;
+  private final SolrMetricManager metricManager;
+  private String registryName;
+
+  /**
+   * Constructs a metric manager.
+   *
+   * @param core the metric manager's core
+   */
+  public SolrCoreMetricManager(SolrCore core) {
+    this.core = core;
+    this.metricManager = core.getCoreDescriptor().getCoreContainer().getMetricManager();
+    registryName = createRegistryName(core.getCoreDescriptor().getCollectionName(), core.getName());
+  }
+
+  /**
+   * Load reporters configured globally and specific to {@link org.apache.solr.core.SolrInfoMBean.Group#core}
+   * group or with a registry name specific to this core.
+   */
+  public void loadReporters() {
+    NodeConfig nodeConfig = core.getCoreDescriptor().getCoreContainer().getConfig();
+    PluginInfo[] pluginInfos = nodeConfig.getMetricReporterPlugins();
+    metricManager.loadReporters(pluginInfos, core.getResourceLoader(), SolrInfoMBean.Group.core, registryName);
+  }
+
+  /**
+   * Make sure that metrics already collected that correspond to the old core name
+   * are carried over and will be used under the new core name.
+   * This method also reloads reporters so that they use the new core name.
+   */
+  public void afterCoreSetName() {
+    String oldRegistryName = registryName;
+    registryName = createRegistryName(core.getCoreDescriptor().getCollectionName(), core.getName());
+    if (oldRegistryName.equals(registryName)) {
+      return;
+    }
+    // close old reporters
+    metricManager.closeReporters(oldRegistryName);
+    metricManager.moveMetrics(oldRegistryName, registryName, null);
+    // old registry is no longer used - we have moved the metrics
+    metricManager.removeRegistry(oldRegistryName);
+    // load reporters again, using the new core name
+    loadReporters();
+  }
+
+  /**
+   * Registers a mapping of name/metric's with the manager's metric registry.
+   *
+   * @param scope     the scope of the metrics to be registered (e.g. `/admin/ping`)
+   * @param producer  producer of metrics to be registered
+   */
+  public void registerMetricProducer(String scope, SolrMetricProducer producer) {
+    if (scope == null || producer == null || producer.getCategory() == null) {
+      throw new IllegalArgumentException("registerMetricProducer() called with illegal arguments: " +
+          "scope = " + scope + ", producer = " + producer);
+    }
+    Collection<String> registered = producer.initializeMetrics(metricManager, getRegistryName(), scope);
+    if (registered == null || registered.isEmpty()) {
+      throw new IllegalArgumentException("registerMetricProducer() did not register any metrics " +
+      "for scope = " + scope + ", producer = " + producer);
+    }
+  }
+
+  /**
+   * Closes reporters specific to this core.
+   */
+  @Override
+  public void close() throws IOException {
+    metricManager.closeReporters(getRegistryName());
+  }
+
+  public SolrCore getCore() {
+    return core;
+  }
+
+  /**
+   * Retrieves the metric registry name of the manager.
+   *
+   * In order to make it easier for reporting tools to aggregate metrics from
+   * different cores that logically belong to a single collection we convert the
+   * core name into a dot-separated hierarchy of: collection name, shard name (with optional split)
+   * and replica name.
+   *
+   * <p>For example, when the core name looks like this but it's NOT a SolrCloud collection:
+   * <code>my_collection_shard1_1_replica1</code> then this will be used as the registry name (plus
+   * the required <code>solr.core</code> prefix). However,
+   * if this is a SolrCloud collection <code>my_collection</code> then the registry name will become
+   * <code>solr.core.my_collection.shard1_1.replica1</code>.</p>
+   *
+   *
+   * @return the metric registry name of the manager.
+   */
+  public String getRegistryName() {
+    return registryName;
+  }
+
+  /* package visibility for tests. */
+  String createRegistryName(String collectionName, String coreName) {
+    if (collectionName == null || (collectionName != null && !coreName.startsWith(collectionName + "_"))) {
+      // single core, or unknown naming scheme
+      return SolrMetricManager.getRegistryName(SolrInfoMBean.Group.core, coreName);
+    }
+    // split "collection1_shard1_1_replica1" into parts
+    String str = coreName.substring(collectionName.length() + 1);
+    String shard;
+    String replica = null;
+    int pos = str.lastIndexOf("_replica");
+    if (pos == -1) { // ?? no _replicaN part ??
+      shard = str;
+    } else {
+      shard = str.substring(0, pos);
+      replica = str.substring(pos + 1);
+    }
+    return SolrMetricManager.getRegistryName(SolrInfoMBean.Group.core, collectionName, shard, replica);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/java/org/apache/solr/metrics/SolrMetricInfo.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrMetricInfo.java b/solr/core/src/java/org/apache/solr/metrics/SolrMetricInfo.java
new file mode 100644
index 0000000..f0bc8a1
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrMetricInfo.java
@@ -0,0 +1,104 @@
+/*
+ * 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;
+
+import com.codahale.metrics.MetricRegistry;
+import org.apache.solr.core.SolrInfoMBean;
+
+/**
+ * Wraps meta-data for a metric.
+ */
+public final class SolrMetricInfo {
+  public final String name;
+  public final String scope;
+  public final SolrInfoMBean.Category category;
+
+  /**
+   * Creates a new instance of {@link SolrMetricInfo}.
+   *
+   * @param category the category of the metric (e.g. `QUERYHANDLERS`)
+   * @param scope    the scope of the metric (e.g. `/admin/ping`)
+   * @param name     the name of the metric (e.g. `Requests`)
+   */
+  public SolrMetricInfo(SolrInfoMBean.Category category, String scope, String name) {
+    this.name = name;
+    this.scope = scope;
+    this.category = category;
+  }
+
+  public static SolrMetricInfo of(String fullName) {
+    if (fullName == null || fullName.isEmpty()) {
+      return null;
+    }
+    String[] names = fullName.split("\\.");
+    if (names.length < 3) { // not a valid info
+      return null;
+    }
+    // check top-level name for valid category
+    SolrInfoMBean.Category category;
+    try {
+      category = SolrInfoMBean.Category.valueOf(names[0]);
+    } catch (IllegalArgumentException e) { // not a valid category
+      return null;
+    }
+    String scope = names[1];
+    String name = fullName.substring(names[0].length() + names[1].length() + 2);
+    return new SolrMetricInfo(category, scope, name);
+  }
+
+  /**
+   * Returns the metric name defined by this object.
+   * For example, if the name is `Requests`, scope is `/admin/ping`,
+   * and category is `QUERYHANDLERS`, then the metric name is
+   * `QUERYHANDLERS./admin/ping.Requests`.
+   *
+   * @return the metric name defined by this object
+   */
+  public String getMetricName() {
+    return MetricRegistry.name(category.toString(), scope, name);
+  }
+
+  @Override
+  public String toString() {
+    return "SolrMetricInfo{" +
+        "name='" + name + '\'' +
+        ", scope='" + scope + '\'' +
+        ", category=" + category +
+        '}';
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    SolrMetricInfo that = (SolrMetricInfo) o;
+
+    if (name != null ? !name.equals(that.name) : that.name != null) return false;
+    if (scope != null ? !scope.equals(that.scope) : that.scope != null) return false;
+    return category == that.category;
+
+  }
+
+  @Override
+  public int hashCode() {
+    int result = name != null ? name.hashCode() : 0;
+    result = 31 * result + (scope != null ? scope.hashCode() : 0);
+    result = 31 * result + (category != null ? category.hashCode() : 0);
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java b/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
new file mode 100644
index 0000000..3f51aef
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
@@ -0,0 +1,652 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.Meter;
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricFilter;
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.MetricSet;
+import com.codahale.metrics.SharedMetricRegistries;
+import com.codahale.metrics.Timer;
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.core.SolrInfoMBean;
+import org.apache.solr.core.SolrResourceLoader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class maintains a repository of named {@link MetricRegistry} instances, and provides several
+ * helper methods for managing various aspects of metrics reporting:
+ * <ul>
+ *   <li>registry creation, clearing and removal,</li>
+ *   <li>creation of most common metric implementations,</li>
+ *   <li>management of {@link SolrMetricReporter}-s specific to a named registry.</li>
+ * </ul>
+ * {@link MetricRegistry} instances are automatically created when first referenced by name. Similarly,
+ * instances of {@link Metric} implementations, such as {@link Meter}, {@link Counter}, {@link Timer} and
+ * {@link Histogram} are automatically created and registered under hierarchical names, in a specified
+ * registry, when {@link #meter(String, String, String...)} and other similar methods are called.
+ * <p>This class enforces a common prefix ({@link #REGISTRY_NAME_PREFIX}) in all registry
+ * names.</p>
+ * <p>Solr uses several different registries for collecting metrics belonging to different groups, using
+ * {@link org.apache.solr.core.SolrInfoMBean.Group} as the main name of the registry (plus the
+ * above-mentioned prefix). Instances of {@link SolrMetricManager} are created for each {@link org.apache.solr.core.CoreContainer},
+ * and most registries are local to each instance, with the exception of two global registries:
+ * <code>solr.jetty</code> and <code>solr.jvm</code>, which are shared between all {@link org.apache.solr.core.CoreContainer}-s</p>
+ */
+public class SolrMetricManager {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public static final String REGISTRY_NAME_PREFIX = "solr.";
+
+  public static final String JETTY_REGISTRY = REGISTRY_NAME_PREFIX + SolrInfoMBean.Group.jetty.toString();
+
+  public static final String JVM_REGISTRY = REGISTRY_NAME_PREFIX + SolrInfoMBean.Group.jvm.toString();
+
+  private final ConcurrentMap<String, MetricRegistry> registries = new ConcurrentHashMap<>();
+
+  // these reporters are per CoreContainer
+  private final Map<String, Map<String, SolrMetricReporter>> reporters = new HashMap<>();
+
+  // these reporters are per JVM
+  private static final Map<String, Map<String, SolrMetricReporter>> sharedReporters = new HashMap<>();
+
+  private final Lock reportersLock = new ReentrantLock();
+
+  public SolrMetricManager() { }
+
+  /**
+   * An implementation of {@link MetricFilter} that selects metrics
+   * with names that start with a prefix.
+   */
+  public static class PrefixFilter implements MetricFilter {
+    private final String prefix;
+    private final Set<String> matched = new HashSet<>();
+
+    /**
+     * Create a filter that uses the provided prefix.
+     * @param prefix prefix to use, must not be null. If empty then any
+     *               name will match.
+     */
+    public PrefixFilter(String prefix) {
+      Objects.requireNonNull(prefix);
+      this.prefix = prefix;
+    }
+
+    @Override
+    public boolean matches(String name, Metric metric) {
+      if (prefix.isEmpty()) {
+        matched.add(name);
+        return true;
+      }
+      if (name.startsWith(prefix)) {
+        matched.add(name);
+        return true;
+      } else {
+        return false;
+      }
+    }
+
+    /**
+     * Return the set of names that matched this filter.
+     * @return matching names
+     */
+    public Set<String> getMatched() {
+      return Collections.unmodifiableSet(matched);
+    }
+
+    /**
+     * Clear the set of names that matched.
+     */
+    public void reset() {
+      matched.clear();
+    }
+  }
+
+  /**
+   * Return a set of existing registry names.
+   */
+  public Set<String> registryNames() {
+    Set<String> set = new HashSet<>();
+    set.addAll(registries.keySet());
+    set.addAll(SharedMetricRegistries.names());
+    return Collections.unmodifiableSet(set);
+  }
+
+  /**
+   * Get (or create if not present) a named registry
+   * @param registry name of the registry
+   * @return existing or newly created registry
+   */
+  public MetricRegistry registry(String registry) {
+    registry = overridableRegistryName(registry);
+    if (JETTY_REGISTRY.equals(registry) || JVM_REGISTRY.equals(registry)) {
+      return SharedMetricRegistries.getOrCreate(registry);
+    } else {
+      final MetricRegistry existing = registries.get(registry);
+      if (existing == null) {
+        final MetricRegistry created = new MetricRegistry();
+        final MetricRegistry raced = registries.putIfAbsent(registry, created);
+        if (raced == null) {
+          return created;
+        } else {
+          return raced;
+        }
+      } else {
+        return existing;
+      }
+    }
+  }
+
+  /**
+   * Remove a named registry.
+   * @param registry name of the registry to remove
+   */
+  public void removeRegistry(String registry) {
+    // close any reporters for this registry first
+    closeReporters(registry);
+    // make sure we use a name with prefix, with overrides
+    registry = overridableRegistryName(registry);
+    if (JETTY_REGISTRY.equals(registry) || JVM_REGISTRY.equals(registry)) {
+      SharedMetricRegistries.remove(registry);
+    } else {
+      registries.remove(registry);
+    }
+  }
+
+  /**
+   * Move all matching metrics from one registry to another. This is useful eg. during
+   * {@link org.apache.solr.core.SolrCore} rename or swap operations.
+   * @param fromRegistry source registry
+   * @param toRegistry target registry
+   * @param filter optional {@link MetricFilter} to select what metrics to move. If null
+   *               then all metrics will be moved.
+   */
+  public void moveMetrics(String fromRegistry, String toRegistry, MetricFilter filter) {
+    MetricRegistry from = registry(fromRegistry);
+    MetricRegistry to = registry(toRegistry);
+    if (from == to) {
+      return;
+    }
+    if (filter == null) {
+      to.registerAll(from);
+      from.removeMatching(MetricFilter.ALL);
+    } else {
+      for (Map.Entry<String, Metric> entry : from.getMetrics().entrySet()) {
+        if (filter.matches(entry.getKey(), entry.getValue())) {
+          to.register(entry.getKey(), entry.getValue());
+        }
+      }
+      from.removeMatching(filter);
+    }
+  }
+
+  /**
+   * Register all metrics in the provided {@link MetricSet}, optionally skipping those that
+   * already exist.
+   * @param registry registry name
+   * @param metrics metric set to register
+   * @param force if true then already existing metrics with the same name will be replaced.
+   *                     When false and a metric with the same name already exists an exception
+   *                     will be thrown.
+   * @param metricPath (optional) additional top-most metric name path elements
+   * @throws Exception if a metric with this name already exists.
+   */
+  public void registerAll(String registry, MetricSet metrics, boolean force, String... metricPath) throws Exception {
+    MetricRegistry metricRegistry = registry(registry);
+    synchronized (metricRegistry) {
+      Map<String, Metric> existingMetrics = metricRegistry.getMetrics();
+      for (Map.Entry<String, Metric> entry : metrics.getMetrics().entrySet()) {
+        String fullName = mkName(entry.getKey(), metricPath);
+        if (force && existingMetrics.containsKey(fullName)) {
+          metricRegistry.remove(fullName);
+        }
+        metricRegistry.register(fullName, entry.getValue());
+      }
+    }
+  }
+
+  /**
+   * Remove all metrics from a specified registry.
+   * @param registry registry name
+   */
+  public void clearRegistry(String registry) {
+    registry(registry).removeMatching(MetricFilter.ALL);
+  }
+
+  /**
+   * Remove some metrics from a named registry
+   * @param registry registry name
+   * @param metricPath (optional) top-most metric name path elements. If empty then
+   *        this is equivalent to calling {@link #clearRegistry(String)},
+   *        otherwise non-empty elements will be joined using dotted notation
+   *        to form a fully-qualified prefix. Metrics with names that start
+   *        with the prefix will be removed.
+   * @return set of metrics names that have been removed.
+   */
+  public Set<String> clearMetrics(String registry, String... metricPath) {
+    PrefixFilter filter;
+    if (metricPath == null || metricPath.length == 0) {
+      filter = new PrefixFilter("");
+    } else {
+      String prefix = MetricRegistry.name("", metricPath);
+      filter = new PrefixFilter(prefix);
+    }
+    registry(registry).removeMatching(filter);
+    return filter.getMatched();
+  }
+
+  /**
+   * Create or get an existing named {@link Meter}
+   * @param registry registry name
+   * @param metricName metric name, either final name or a fully-qualified name
+   *                   using dotted notation
+   * @param metricPath (optional) additional top-most metric name path elements
+   * @return existing or a newly created {@link Meter}
+   */
+  public Meter meter(String registry, String metricName, String... metricPath) {
+    return registry(registry).meter(mkName(metricName, metricPath));
+  }
+
+  /**
+   * Create or get an existing named {@link Timer}
+   * @param registry registry name
+   * @param metricName metric name, either final name or a fully-qualified name
+   *                   using dotted notation
+   * @param metricPath (optional) additional top-most metric name path elements
+   * @return existing or a newly created {@link Timer}
+   */
+  public Timer timer(String registry, String metricName, String... metricPath) {
+    return registry(registry).timer(mkName(metricName, metricPath));
+  }
+
+  /**
+   * Create or get an existing named {@link Counter}
+   * @param registry registry name
+   * @param metricName metric name, either final name or a fully-qualified name
+   *                   using dotted notation
+   * @param metricPath (optional) additional top-most metric name path elements
+   * @return existing or a newly created {@link Counter}
+   */
+  public Counter counter(String registry, String metricName, String... metricPath) {
+    return registry(registry).counter(mkName(metricName, metricPath));
+  }
+
+  /**
+   * Create or get an existing named {@link Histogram}
+   * @param registry registry name
+   * @param metricName metric name, either final name or a fully-qualified name
+   *                   using dotted notation
+   * @param metricPath (optional) additional top-most metric name path elements
+   * @return existing or a newly created {@link Histogram}
+   */
+  public Histogram histogram(String registry, String metricName, String... metricPath) {
+    return registry(registry).histogram(mkName(metricName, metricPath));
+  }
+
+  /**
+   * Register an instance of {@link Metric}.
+   * @param registry registry name
+   * @param metric metric instance
+   * @param force if true then an already existing metric with the same name will be replaced.
+   *                     When false and a metric with the same name already exists an exception
+   *                     will be thrown.
+   * @param metricName metric name, either final name or a fully-qualified name
+   *                   using dotted notation
+   * @param metricPath (optional) additional top-most metric name path elements
+   */
+  public void register(String registry, Metric metric, boolean force, String metricName, String... metricPath) {
+    MetricRegistry metricRegistry = registry(registry);
+    String fullName = mkName(metricName, metricPath);
+    synchronized (metricRegistry) {
+      if (force && metricRegistry.getMetrics().containsKey(fullName)) {
+        metricRegistry.remove(fullName);
+      }
+      metricRegistry.register(fullName, metric);
+    }
+  }
+
+
+
+  /**
+   * This method creates a hierarchical name with arbitrary levels of hierarchy
+   * @param name the final segment of the name, must not be null or empty.
+   * @param path optional path segments, starting from the top level. Empty or null
+   *             segments will be skipped.
+   * @return fully-qualified name using dotted notation, with all valid hierarchy
+   * segments prepended to the name.
+   */
+  public static String mkName(String name, String... path) {
+    if (name == null || name.isEmpty()) {
+      throw new IllegalArgumentException("name must not be empty");
+    }
+    if (path == null || path.length == 0) {
+      return name;
+    } else {
+      StringBuilder sb = new StringBuilder();
+      for (String s : path) {
+        if (s == null || s.isEmpty()) {
+          continue;
+        }
+        if (sb.length() > 0) {
+          sb.append('.');
+        }
+        sb.append(s);
+      }
+      if (sb.length() > 0) {
+        sb.append('.');
+      }
+      sb.append(name);
+      return sb.toString();
+    }
+  }
+
+  /**
+   * Allows named registries to be renamed using System properties.
+   * This would be mostly be useful if you want to combine the metrics from a few registries for a single
+   * reporter.
+   * <p>For example, in order to collect metrics from related cores in a single registry you could specify
+   * the following system properties:</p>
+   * <pre>
+   *   ... -Dsolr.core.collection1=solr.core.allCollections -Dsolr.core.collection2=solr.core.allCollections
+   * </pre>
+   * <b>NOTE:</b> Once a registry is renamed in a way that its metrics are combined with another repository
+   * it is no longer possible to retrieve the original metrics until this renaming is removed and the Solr
+   * {@link org.apache.solr.core.SolrInfoMBean.Group} of components that reported to that name is restarted.
+   * @param registry The name of the registry
+   * @return A potentially overridden (via System properties) registry name
+   */
+  public static String overridableRegistryName(String registry) {
+    String fqRegistry = enforcePrefix(registry);
+    return enforcePrefix(System.getProperty(fqRegistry,fqRegistry));
+  }
+
+  /**
+   * Enforces the leading {@link #REGISTRY_NAME_PREFIX} in a name.
+   * @param name input name, possibly without the prefix
+   * @return original name if it contained the prefix, or the
+   * input name with the prefix prepended.
+   */
+  public static String enforcePrefix(String name) {
+    if (name.startsWith(REGISTRY_NAME_PREFIX)) {
+      return name;
+    } else {
+      return new StringBuilder(REGISTRY_NAME_PREFIX).append(name).toString();
+    }
+  }
+
+  /**
+   * Helper method to construct a properly prefixed registry name based on the group.
+   * @param group reporting group
+   * @param names optional child elements of the registry name. If exactly one element is provided
+   *              and it already contains the required prefix and group name then this value will be used,
+   *              and the group parameter will be ignored.
+   * @return fully-qualified and prefixed registry name, with overrides applied.
+   */
+  public static String getRegistryName(SolrInfoMBean.Group group, String... names) {
+    String fullName;
+    String prefix = REGISTRY_NAME_PREFIX + group.toString() + ".";
+    // check for existing prefix and group
+    if (names != null && names.length > 0 && names[0] != null && names[0].startsWith(prefix)) {
+      // assume the first segment already was expanded
+      if (names.length > 1) {
+        String[] newNames = new String[names.length - 1];
+        System.arraycopy(names, 1, newNames, 0, newNames.length);
+        fullName = MetricRegistry.name(names[0], newNames);
+      } else {
+        fullName = MetricRegistry.name(names[0]);
+      }
+    } else {
+      fullName = MetricRegistry.name(group.toString(), names);
+    }
+    return overridableRegistryName(fullName);
+  }
+
+  // reporter management
+
+  /**
+   * Create and register {@link SolrMetricReporter}-s specific to a {@link org.apache.solr.core.SolrInfoMBean.Group}.
+   * Note: reporters that specify neither "group" nor "registry" attributes are treated as universal -
+   * they will always be loaded for any group. These two attributes may also contain multiple comma- or
+   * whitespace-separated values, in which case the reporter will be loaded for any matching value from
+   * the list. If both attributes are present then only "group" attribute will be processed.
+   * @param pluginInfos plugin configurations
+   * @param loader resource loader
+   * @param group selected group, not null
+   * @param registryNames optional child registry name elements
+   */
+  public void loadReporters(PluginInfo[] pluginInfos, SolrResourceLoader loader, SolrInfoMBean.Group group, String... registryNames) {
+    if (pluginInfos == null || pluginInfos.length == 0) {
+      return;
+    }
+    String registryName = getRegistryName(group, registryNames);
+    for (PluginInfo info : pluginInfos) {
+      String target = info.attributes.get("group");
+      if (target == null) { // no "group"
+        target = info.attributes.get("registry");
+        if (target != null) {
+          String[] targets = target.split("[\\s,]+");
+          boolean found = false;
+          for (String t : targets) {
+            t = overridableRegistryName(t);
+            if (registryName.equals(t)) {
+              found = true;
+              break;
+            }
+          }
+          if (!found) {
+            continue;
+          }
+        } else {
+          // neither group nor registry specified.
+          // always register this plugin for all groups and registries
+        }
+      } else { // check groups
+        String[] targets = target.split("[\\s,]+");
+        boolean found = false;
+        for (String t : targets) {
+          if (group.toString().equals(t)) {
+            found = true;
+            break;
+          }
+        }
+        if (!found) {
+          continue;
+        }
+      }
+      try {
+        loadReporter(registryName, loader, info);
+      } catch (Exception e) {
+        log.warn("Error loading metrics reporter, plugin info: " + info, e);
+      }
+    }
+  }
+
+  /**
+   * Create and register an instance of {@link SolrMetricReporter}.
+   * @param registry reporter is associated with this registry
+   * @param loader loader to use when creating an instance of the reporter
+   * @param pluginInfo plugin configuration. Plugin "name" and "class" attributes are required.
+   * @throws Exception if any argument is missing or invalid
+   */
+  public void loadReporter(String registry, SolrResourceLoader loader, PluginInfo pluginInfo) throws Exception {
+    if (registry == null || pluginInfo == null || pluginInfo.name == null || pluginInfo.className == null) {
+      throw new IllegalArgumentException("loadReporter called with missing arguments: " +
+          "registry=" + registry + ", loader=" + loader + ", pluginInfo=" + pluginInfo);
+    }
+    // make sure we use a name with prefix, with overrides
+    registry = overridableRegistryName(registry);
+    SolrMetricReporter reporter = loader.newInstance(
+        pluginInfo.className,
+        SolrMetricReporter.class,
+        new String[0],
+        new Class[] { SolrMetricManager.class, String.class },
+        new Object[] { this, registry }
+    );
+    try {
+      reporter.init(pluginInfo);
+    } catch (IllegalStateException e) {
+      throw new IllegalArgumentException("reporter init failed: " + pluginInfo, e);
+    }
+    try {
+      if (!reportersLock.tryLock(10, TimeUnit.SECONDS)) {
+        throw new Exception("Could not obtain lock to modify reporters registry: " + registry);
+      }
+    } catch (InterruptedException e) {
+      throw new Exception("Interrupted while trying to obtain lock to modify reporters registry: " + registry);
+    }
+    try {
+      Map<String, SolrMetricReporter> perRegistry = reporters.get(registry);
+      if (perRegistry == null) {
+        perRegistry = new HashMap<>();
+        reporters.put(registry, perRegistry);
+      }
+      SolrMetricReporter oldReporter = perRegistry.get(pluginInfo.name);
+      if (oldReporter != null) { // close it
+        log.info("Replacing existing reporter '" + pluginInfo.name + "' in registry '" + registry + "': " + oldReporter.toString());
+        oldReporter.close();
+      }
+      perRegistry.put(pluginInfo.name, reporter);
+
+    } finally {
+      reportersLock.unlock();
+    }
+  }
+
+  /**
+   * Close and unregister a named {@link SolrMetricReporter} for a registry.
+   * @param registry registry name
+   * @param name reporter name
+   * @return true if a named reporter existed and was closed.
+   */
+  public boolean closeReporter(String registry, String name) {
+    // make sure we use a name with prefix, with overrides
+    registry = overridableRegistryName(registry);
+    try {
+      if (!reportersLock.tryLock(10, TimeUnit.SECONDS)) {
+        log.warn("Could not obtain lock to modify reporters registry: " + registry);
+        return false;
+      }
+    } catch (InterruptedException e) {
+      log.warn("Interrupted while trying to obtain lock to modify reporters registry: " + registry);
+      return false;
+    }
+    try {
+      Map<String, SolrMetricReporter> perRegistry = reporters.get(registry);
+      if (perRegistry == null) {
+        return false;
+      }
+      SolrMetricReporter reporter = perRegistry.remove(name);
+      if (reporter == null) {
+        return false;
+      }
+      try {
+        reporter.close();
+      } catch (Exception e) {
+        log.warn("Error closing metric reporter, registry=" + registry + ", name=" + name, e);
+      }
+      return true;
+    } finally {
+      reportersLock.unlock();
+    }
+  }
+
+  /**
+   * Close and unregister all {@link SolrMetricReporter}-s for a registry.
+   * @param registry registry name
+   * @return names of closed reporters
+   */
+  public Set<String> closeReporters(String registry) {
+    // make sure we use a name with prefix, with overrides
+    registry = overridableRegistryName(registry);
+    try {
+      if (!reportersLock.tryLock(10, TimeUnit.SECONDS)) {
+        log.warn("Could not obtain lock to modify reporters registry: " + registry);
+        return Collections.emptySet();
+      }
+    } catch (InterruptedException e) {
+      log.warn("Interrupted while trying to obtain lock to modify reporters registry: " + registry);
+      return Collections.emptySet();
+    }
+    log.info("Closing metric reporters for: " + registry);
+    try {
+      Map<String, SolrMetricReporter> perRegistry = reporters.remove(registry);
+      if (perRegistry != null) {
+        for (SolrMetricReporter reporter : perRegistry.values()) {
+          try {
+            reporter.close();
+          } catch (IOException ioe) {
+            log.warn("Exception closing reporter " + reporter, ioe);
+          }
+        }
+        return perRegistry.keySet();
+      } else {
+        return Collections.emptySet();
+      }
+    } finally {
+      reportersLock.unlock();
+    }
+  }
+
+  /**
+   * Get a map of reporters for a registry. Keys are reporter names, values are reporter instances.
+   * @param registry registry name
+   * @return map of reporters and their names, may be empty but never null
+   */
+  public Map<String, SolrMetricReporter> getReporters(String registry) {
+    // make sure we use a name with prefix, with overrides
+    registry = overridableRegistryName(registry);
+    try {
+      if (!reportersLock.tryLock(10, TimeUnit.SECONDS)) {
+        log.warn("Could not obtain lock to modify reporters registry: " + registry);
+        return Collections.emptyMap();
+      }
+    } catch (InterruptedException e) {
+      log.warn("Interrupted while trying to obtain lock to modify reporters registry: " + registry);
+      return Collections.emptyMap();
+    }
+    try {
+      Map<String, SolrMetricReporter> perRegistry = reporters.get(registry);
+      if (perRegistry == null) {
+        return Collections.emptyMap();
+      } else {
+        // defensive copy - the original map may change after we release the lock
+        return Collections.unmodifiableMap(new HashMap<>(perRegistry));
+      }
+    } finally {
+      reportersLock.unlock();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/java/org/apache/solr/metrics/SolrMetricProducer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrMetricProducer.java b/solr/core/src/java/org/apache/solr/metrics/SolrMetricProducer.java
new file mode 100644
index 0000000..472931f
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrMetricProducer.java
@@ -0,0 +1,38 @@
+/*
+ * 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;
+
+import java.util.Collection;
+
+import org.apache.solr.core.SolrInfoMBean;
+
+/**
+ * Extension of {@link SolrInfoMBean} for use by objects that
+ * expose metrics through {@link SolrCoreMetricManager}.
+ */
+public interface SolrMetricProducer extends SolrInfoMBean {
+
+  /**
+   * Initializes metrics specific to this producer
+   * @param manager an instance of {@link SolrMetricManager}
+   * @param registry registry name where metrics are registered
+   * @param scope scope of the metrics (eg. handler name) to separate metrics of
+   *              instances of the same component executing in different contexts
+   * @return registered (or existing) unqualified names of metrics specific to this producer.
+   */
+  Collection<String> initializeMetrics(SolrMetricManager manager, String registry, String scope);
+}


[29/44] lucene-solr:jira/solr-9854: SOLR-9513: Generic Hadoop authentication plugins, GenericHadoopAuthPlugin and ConfigurableInternodeAuthHadoopPlugin

Posted by ab...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/core/src/test/org/apache/solr/security/hadoop/TestImpersonationWithHadoopAuth.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/hadoop/TestImpersonationWithHadoopAuth.java b/solr/core/src/test/org/apache/solr/security/hadoop/TestImpersonationWithHadoopAuth.java
new file mode 100644
index 0000000..ed8397b
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/security/hadoop/TestImpersonationWithHadoopAuth.java
@@ -0,0 +1,215 @@
+/*
+ * 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.security.hadoop;
+
+import static org.apache.solr.security.HttpParamDelegationTokenPlugin.USER_PARAM;
+import static org.apache.solr.security.hadoop.ImpersonationUtil.*;
+
+import java.net.InetAddress;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.util.Constants;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.security.HadoopAuthPlugin;
+import org.apache.solr.servlet.SolrRequestParsers;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestImpersonationWithHadoopAuth  extends SolrCloudTestCase {
+  protected static final int NUM_SERVERS = 2;
+  private static final boolean defaultAddRequestHeadersToContext =
+      SolrRequestParsers.DEFAULT.isAddRequestHeadersToContext();
+
+  @SuppressWarnings("unchecked")
+  @BeforeClass
+  public static void setupClass() throws Exception {
+    assumeFalse("Hadoop does not work on Windows", Constants.WINDOWS);
+
+    InetAddress loopback = InetAddress.getLoopbackAddress();
+    Path securityJsonPath = TEST_PATH().resolve("security").resolve("hadoop_simple_auth_with_delegation.json");
+    String securityJson = new String(Files.readAllBytes(securityJsonPath), Charset.defaultCharset());
+
+    Map<String, Object> securityConfig = (Map<String, Object>)Utils.fromJSONString(securityJson);
+    Map<String, Object> authConfig = (Map<String, Object>)securityConfig.get("authentication");
+    Map<String,String> proxyUserConfigs = (Map<String,String>) authConfig
+        .getOrDefault(HadoopAuthPlugin.PROXY_USER_CONFIGS, new HashMap<>());
+    proxyUserConfigs.put("proxyuser.noGroups.hosts", "*");
+    proxyUserConfigs.put("proxyuser.anyHostAnyUser.hosts", "*");
+    proxyUserConfigs.put("proxyuser.anyHostAnyUser.groups", "*");
+    proxyUserConfigs.put("proxyuser.wrongHost.hosts", "1.1.1.1.1.1");
+    proxyUserConfigs.put("proxyuser.wrongHost.groups", "*");
+    proxyUserConfigs.put("proxyuser.noHosts.groups", "*");
+    proxyUserConfigs.put("proxyuser.localHostAnyGroup.hosts",
+        loopback.getCanonicalHostName() + "," + loopback.getHostName() + "," + loopback.getHostAddress());
+    proxyUserConfigs.put("proxyuser.localHostAnyGroup.groups", "*");
+    proxyUserConfigs.put("proxyuser.bogusGroup.hosts", "*");
+    proxyUserConfigs.put("proxyuser.bogusGroup.groups", "__some_bogus_group");
+    proxyUserConfigs.put("proxyuser.anyHostUsersGroup.groups", ImpersonationUtil.getUsersFirstGroup());
+    proxyUserConfigs.put("proxyuser.anyHostUsersGroup.hosts", "*");
+
+    authConfig.put(HadoopAuthPlugin.PROXY_USER_CONFIGS, proxyUserConfigs);
+
+    SolrRequestParsers.DEFAULT.setAddRequestHeadersToContext(true);
+    System.setProperty("collectionsHandler", ImpersonatorCollectionsHandler.class.getName());
+
+    configureCluster(NUM_SERVERS)// nodes
+        .withSecurityJson(Utils.toJSONString(securityConfig))
+        .addConfig("conf1", TEST_PATH().resolve("configsets").resolve("cloud-minimal").resolve("conf"))
+        .configure();
+  }
+
+  @AfterClass
+  public static void tearDownClass() throws Exception {
+    SolrRequestParsers.DEFAULT.setAddRequestHeadersToContext(defaultAddRequestHeadersToContext);
+    System.clearProperty("collectionsHandler");
+  }
+
+  private SolrClient newSolrClient() {
+    return new HttpSolrClient.Builder(
+        cluster.getJettySolrRunner(0).getBaseUrl().toString()).build();
+  }
+
+  @Test
+  public void testProxyNoConfigGroups() throws Exception {
+    try (SolrClient solrClient = newSolrClient()) {
+      solrClient.request(getProxyRequest("noGroups","bar"));
+      fail("Expected RemoteSolrException");
+    }
+    catch (HttpSolrClient.RemoteSolrException ex) {
+      assertTrue(ex.getLocalizedMessage(), ex.getMessage().contains(getExpectedGroupExMsg("noGroups", "bar")));
+    }
+  }
+
+  @Test
+  public void testProxyWrongHost() throws Exception {
+    try (SolrClient solrClient = newSolrClient()) {
+      solrClient.request(getProxyRequest("wrongHost","bar"));
+      fail("Expected RemoteSolrException");
+    }
+    catch (HttpSolrClient.RemoteSolrException ex) {
+      assertTrue(ex.getMessage().contains(getExpectedHostExMsg("wrongHost")));
+    }
+  }
+
+  @Test
+  public void testProxyNoConfigHosts() throws Exception {
+    try (SolrClient solrClient = newSolrClient()) {
+      solrClient.request(getProxyRequest("noHosts","bar"));
+      fail("Expected RemoteSolrException");
+    }
+    catch (HttpSolrClient.RemoteSolrException ex) {
+      assertTrue(ex.getMessage().contains(getExpectedHostExMsg("noHosts")));
+    }
+  }
+
+  @Test
+  public void testProxyValidateAnyHostAnyUser() throws Exception {
+    try (SolrClient solrClient = newSolrClient()) {
+      solrClient.request(getProxyRequest("anyHostAnyUser", "bar"));
+      assertTrue(ImpersonatorCollectionsHandler.called.get());
+    }
+  }
+
+  @Test
+  public void testProxyInvalidProxyUser() throws Exception {
+    try (SolrClient solrClient = newSolrClient()) {
+      // wrong direction, should fail
+      solrClient.request(getProxyRequest("bar","anyHostAnyUser"));
+      fail("Expected RemoteSolrException");
+    }
+    catch (HttpSolrClient.RemoteSolrException ex) {
+      assertTrue(ex.getMessage().contains(getExpectedGroupExMsg("bar", "anyHostAnyUser")));
+    }
+  }
+
+  @Test
+  public void testProxyValidateHost() throws Exception {
+    try (SolrClient solrClient = newSolrClient()) {
+      solrClient.request(getProxyRequest("localHostAnyGroup", "bar"));
+      assertTrue(ImpersonatorCollectionsHandler.called.get());
+    }
+  }
+
+  @Test
+  public void testProxyValidateGroup() throws Exception {
+    try (SolrClient solrClient = newSolrClient()) {
+      solrClient.request(getProxyRequest("anyHostUsersGroup", System.getProperty("user.name")));
+      assertTrue(ImpersonatorCollectionsHandler.called.get());
+    }
+  }
+
+  @Test
+  public void testProxyInvalidGroup() throws Exception {
+    try (SolrClient solrClient = newSolrClient()) {
+      solrClient.request(getProxyRequest("bogusGroup","bar"));
+      fail("Expected RemoteSolrException");
+    }
+    catch (HttpSolrClient.RemoteSolrException ex) {
+      assertTrue(ex.getMessage().contains(getExpectedGroupExMsg("bogusGroup", "bar")));
+    }
+  }
+
+  @Test
+  public void testProxyNullProxyUser() throws Exception {
+    try (SolrClient solrClient = newSolrClient()) {
+      solrClient.request(getProxyRequest("","bar"));
+      fail("Expected RemoteSolrException");
+    }
+    catch (HttpSolrClient.RemoteSolrException ex) {
+      // this exception is specific to our implementation, don't check a specific message.
+    }
+  }
+
+  @Test
+  @AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/HADOOP-9893")
+  public void testForwarding() throws Exception {
+    String collectionName = "forwardingCollection";
+
+    // create collection
+    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName, "conf1",
+        1, 1);
+    try (SolrClient solrClient = newSolrClient()) {
+      create.process(solrClient);
+    }
+
+    // try a command to each node, one of them must be forwarded
+    for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
+      HttpSolrClient client =
+          new HttpSolrClient.Builder(jetty.getBaseUrl().toString() + "/" + collectionName).build();
+      try {
+        ModifiableSolrParams params = new ModifiableSolrParams();
+        params.set("q", "*:*");
+        params.set(USER_PARAM, "user");
+        client.query(params);
+      } finally {
+        client.close();
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/core/src/test/org/apache/solr/security/hadoop/TestSolrCloudWithHadoopAuthPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/hadoop/TestSolrCloudWithHadoopAuthPlugin.java b/solr/core/src/test/org/apache/solr/security/hadoop/TestSolrCloudWithHadoopAuthPlugin.java
new file mode 100644
index 0000000..960fd9a
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/security/hadoop/TestSolrCloudWithHadoopAuthPlugin.java
@@ -0,0 +1,136 @@
+/*
+ * 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.security.hadoop;
+
+import java.io.File;
+import java.nio.charset.StandardCharsets;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.lucene.util.Constants;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.cloud.AbstractDistribZkTestBase;
+import org.apache.solr.cloud.KerberosTestServices;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.SolrInputDocument;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestSolrCloudWithHadoopAuthPlugin extends SolrCloudTestCase {
+  protected static final int NUM_SERVERS = 1;
+  protected static final int NUM_SHARDS = 1;
+  protected static final int REPLICATION_FACTOR = 1;
+  private static KerberosTestServices kerberosTestServices;
+
+  @BeforeClass
+  public static void setupClass() throws Exception {
+    assumeFalse("Hadoop does not work on Windows", Constants.WINDOWS);
+    assumeFalse("FIXME: SOLR-8182: This test fails under Java 9", Constants.JRE_IS_MINIMUM_JAVA9);
+
+    setupMiniKdc();
+
+    configureCluster(NUM_SERVERS)// nodes
+        .withSecurityJson(TEST_PATH().resolve("security").resolve("hadoop_kerberos_config.json"))
+        .addConfig("conf1", TEST_PATH().resolve("configsets").resolve("cloud-minimal").resolve("conf"))
+        .configure();
+  }
+
+  @AfterClass
+  public static void tearDownClass() throws Exception {
+    System.clearProperty("java.security.auth.login.config");
+    System.clearProperty("solr.kerberos.principal");
+    System.clearProperty("solr.kerberos.keytab");
+    System.clearProperty("solr.kerberos.name.rules");
+    System.clearProperty("solr.jaas.debug");
+    kerberosTestServices.stop();
+    kerberosTestServices = null;
+  }
+
+  private static void setupMiniKdc() throws Exception {
+    System.setProperty("solr.jaas.debug", "true");
+    String kdcDir = createTempDir()+File.separator+"minikdc";
+    String solrClientPrincipal = "solr";
+    File keytabFile = new File(kdcDir, "keytabs");
+    kerberosTestServices = KerberosTestServices.builder()
+        .withKdc(new File(kdcDir))
+        .withJaasConfiguration(solrClientPrincipal, keytabFile, "SolrClient")
+        .build();
+    String solrServerPrincipal = "HTTP/127.0.0.1";
+    kerberosTestServices.start();
+    kerberosTestServices.getKdc().createPrincipal(keytabFile, solrServerPrincipal, solrClientPrincipal);
+
+    String jaas = "SolrClient {\n"
+        + " com.sun.security.auth.module.Krb5LoginModule required\n"
+        + " useKeyTab=true\n"
+        + " keyTab=\"" + keytabFile.getAbsolutePath() + "\"\n"
+        + " storeKey=true\n"
+        + " useTicketCache=false\n"
+        + " doNotPrompt=true\n"
+        + " debug=true\n"
+        + " principal=\"" + solrClientPrincipal + "\";\n"
+        + "};";
+
+    String jaasFilePath = kdcDir+File.separator+"jaas-client.conf";
+    FileUtils.write(new File(jaasFilePath), jaas, StandardCharsets.UTF_8);
+    System.setProperty("java.security.auth.login.config", jaasFilePath);
+    System.setProperty("solr.kerberos.jaas.appname", "SolrClient"); // Get this app name from the jaas file
+
+    System.setProperty("solr.kerberos.principal", solrServerPrincipal);
+    System.setProperty("solr.kerberos.keytab", keytabFile.getAbsolutePath());
+    // Extracts 127.0.0.1 from HTTP/127.0.0.1@EXAMPLE.COM
+    System.setProperty("solr.kerberos.name.rules", "RULE:[1:$1@$0](.*EXAMPLE.COM)s/@.*//"
+        + "\nRULE:[2:$2@$0](.*EXAMPLE.COM)s/@.*//"
+        + "\nDEFAULT"
+        );
+  }
+
+  @Test
+  public void testBasics() throws Exception {
+    testCollectionCreateSearchDelete();
+    // sometimes run a second test e.g. to test collection create-delete-create scenario
+    if (random().nextBoolean()) testCollectionCreateSearchDelete();
+  }
+
+  protected void testCollectionCreateSearchDelete() throws Exception {
+    CloudSolrClient solrClient = cluster.getSolrClient();
+    String collectionName = "testkerberoscollection";
+
+    // create collection
+    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName, "conf1",
+        NUM_SHARDS, REPLICATION_FACTOR);
+    create.process(solrClient);
+
+    SolrInputDocument doc = new SolrInputDocument();
+    doc.setField("id", "1");
+    solrClient.add(collectionName, doc);
+    solrClient.commit(collectionName);
+
+    SolrQuery query = new SolrQuery();
+    query.setQuery("*:*");
+    QueryResponse rsp = solrClient.query(collectionName, query);
+    assertEquals(1, rsp.getResults().getNumFound());
+
+    CollectionAdminRequest.Delete deleteReq = CollectionAdminRequest.deleteCollection(collectionName);
+    deleteReq.process(solrClient);
+    AbstractDistribZkTestBase.waitForCollectionToDisappear(collectionName,
+        solrClient.getZkStateReader(), true, true, 330);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClientBuilderFactory.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClientBuilderFactory.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClientBuilderFactory.java
new file mode 100644
index 0000000..77c4a94
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClientBuilderFactory.java
@@ -0,0 +1,41 @@
+/*
+ * 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.client.solrj.impl;
+
+import java.io.Closeable;
+import java.util.Optional;
+
+/**
+ * Factory interface for configuring {@linkplain SolrHttpClientBuilder}. This
+ * relies on the internal HttpClient implementation and is subject to
+ * change.
+ *
+ * @lucene.experimental
+ **/
+public interface HttpClientBuilderFactory extends Closeable {
+
+  /**
+   * This method configures the {@linkplain SolrHttpClientBuilder} by overriding the
+   * configuration of passed SolrHttpClientBuilder or as a new instance.
+   *
+   * @param builder The instance of the {@linkplain SolrHttpClientBuilder} which should
+   *                by configured (optional).
+   * @return the {@linkplain SolrHttpClientBuilder}
+   */
+  public SolrHttpClientBuilder getHttpClientBuilder(Optional<SolrHttpClientBuilder> builder);
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Krb5HttpClientBuilder.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Krb5HttpClientBuilder.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Krb5HttpClientBuilder.java
index 39cc2dc..7f3cf29 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Krb5HttpClientBuilder.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Krb5HttpClientBuilder.java
@@ -21,6 +21,7 @@ import java.security.Principal;
 import java.util.Arrays;
 import java.util.HashSet;
 import java.util.Locale;
+import java.util.Optional;
 import java.util.Set;
 
 import javax.security.auth.login.AppConfigurationEntry;
@@ -46,7 +47,7 @@ import org.slf4j.LoggerFactory;
 /**
  * Kerberos-enabled SolrHttpClientBuilder
  */
-public class Krb5HttpClientBuilder  {
+public class Krb5HttpClientBuilder implements HttpClientBuilderFactory {
   
   public static final String LOGIN_CONFIG_PROP = "java.security.auth.login.config";
   private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@@ -73,7 +74,12 @@ public class Krb5HttpClientBuilder  {
   public void close() {
     HttpClientUtil.removeRequestInterceptor(bufferedEntityInterceptor);
   }
-  
+
+  @Override
+  public SolrHttpClientBuilder getHttpClientBuilder(Optional<SolrHttpClientBuilder> builder) {
+    return builder.isPresent() ? getBuilder(builder.get()) : getBuilder();
+  }
+
   public SolrHttpClientBuilder getBuilder(SolrHttpClientBuilder builder) {
     if (System.getProperty(LOGIN_CONFIG_PROP) != null) {
       String configValue = System.getProperty(LOGIN_CONFIG_PROP);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java b/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
index 5ebdfb7..81e1f22 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
@@ -28,6 +28,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.Objects;
+import java.util.Optional;
 import java.util.Random;
 import java.util.SortedMap;
 import java.util.concurrent.Callable;
@@ -68,6 +69,7 @@ public class MiniSolrCloudCluster {
       "  <str name=\"shareSchema\">${shareSchema:false}</str>\n" +
       "  <str name=\"configSetBaseDir\">${configSetBaseDir:configsets}</str>\n" +
       "  <str name=\"coreRootDirectory\">${coreRootDirectory:.}</str>\n" +
+      "  <str name=\"collectionsHandler\">${collectionsHandler:solr.CollectionsHandler}</str>\n" +
       "\n" +
       "  <shardHandlerFactory name=\"shardHandlerFactory\" class=\"HttpShardHandlerFactory\">\n" +
       "    <str name=\"urlScheme\">${urlScheme:}</str>\n" +
@@ -180,8 +182,30 @@ public class MiniSolrCloudCluster {
    *
    * @throws Exception if there was an error starting the cluster
    */
-  public MiniSolrCloudCluster(int numServers, Path baseDir, String solrXml, JettyConfig jettyConfig, ZkTestServer zkTestServer) throws Exception {
+  public MiniSolrCloudCluster(int numServers, Path baseDir, String solrXml, JettyConfig jettyConfig,
+      ZkTestServer zkTestServer) throws Exception {
+    this(numServers, baseDir, solrXml, jettyConfig, zkTestServer, Optional.empty());
+  }
 
+  /**
+   * Create a MiniSolrCloudCluster.
+   * Note - this constructor visibility is changed to package protected so as to
+   * discourage its usage. Ideally *new* functionality should use {@linkplain SolrCloudTestCase}
+   * to configure any additional parameters.
+   *
+   * @param numServers number of Solr servers to start
+   * @param baseDir base directory that the mini cluster should be run from
+   * @param solrXml solr.xml file to be uploaded to ZooKeeper
+   * @param jettyConfig Jetty configuration
+   * @param zkTestServer ZkTestServer to use.  If null, one will be created
+   * @param securityJson A string representation of security.json file (optional).
+   *
+   * @throws Exception if there was an error starting the cluster
+   */
+   MiniSolrCloudCluster(int numServers, Path baseDir, String solrXml, JettyConfig jettyConfig,
+      ZkTestServer zkTestServer, Optional<String> securityJson) throws Exception {
+
+    Objects.requireNonNull(securityJson);
     this.baseDir = Objects.requireNonNull(baseDir);
     this.jettyConfig = Objects.requireNonNull(jettyConfig);
 
@@ -202,6 +226,9 @@ public class MiniSolrCloudCluster {
       if (jettyConfig.sslConfig != null && jettyConfig.sslConfig.isSSLMode()) {
         zkClient.makePath("/solr" + ZkStateReader.CLUSTER_PROPS, "{'urlScheme':'https'}".getBytes(StandardCharsets.UTF_8), true);
       }
+      if (securityJson.isPresent()) { // configure Solr security
+        zkClient.makePath("/solr/security.json", securityJson.get().getBytes(Charset.defaultCharset()), true);
+      }
     }
 
     // tell solr to look in zookeeper for solr.xml

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java b/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java
index 9cc4a22..34dc8ac 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java
@@ -26,6 +26,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.Predicate;
@@ -90,6 +91,7 @@ public class SolrCloudTestCase extends SolrTestCaseJ4 {
     private final Path baseDir;
     private String solrxml = MiniSolrCloudCluster.DEFAULT_CLOUD_SOLR_XML;
     private JettyConfig jettyConfig = buildJettyConfig("/solr");
+    private Optional<String> securityJson = Optional.empty();
 
     private List<Config> configs = new ArrayList<>();
     private Map<String, String> clusterProperties = new HashMap<>();
@@ -133,6 +135,32 @@ public class SolrCloudTestCase extends SolrTestCaseJ4 {
     }
 
     /**
+     * Configure the specified security.json for the {@linkplain MiniSolrCloudCluster}
+     *
+     * @param securityJson The path specifying the security.json file
+     * @return the instance of {@linkplain Builder}
+     */
+    public Builder withSecurityJson(Path securityJson) {
+      try {
+        this.securityJson = Optional.of(new String(Files.readAllBytes(securityJson), Charset.defaultCharset()));
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+      return this;
+    }
+
+    /**
+     * Configure the specified security.json for the {@linkplain MiniSolrCloudCluster}
+     *
+     * @param securityJson The string specifying the security.json configuration
+     * @return the instance of {@linkplain Builder}
+     */
+    public Builder withSecurityJson(String securityJson) {
+      this.securityJson = Optional.of(securityJson);
+      return this;
+    }
+
+    /**
      * Upload a collection config before tests start
      * @param configName the config name
      * @param configPath the path to the config files
@@ -157,7 +185,7 @@ public class SolrCloudTestCase extends SolrTestCaseJ4 {
      * @throws Exception if an error occurs on startup
      */
     public void configure() throws Exception {
-      cluster = new MiniSolrCloudCluster(nodeCount, baseDir, solrxml, jettyConfig);
+      cluster = new MiniSolrCloudCluster(nodeCount, baseDir, solrxml, jettyConfig, null, securityJson);
       CloudSolrClient client = cluster.getSolrClient();
       for (Config config : configs) {
         ((ZkClientClusterStateProvider)client.getClusterStateProvider()).uploadConfig(config.path, config.name);


[21/44] lucene-solr:jira/solr-9854: null the static fields after test finishes

Posted by ab...@apache.org.
null the static fields after test finishes


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

Branch: refs/heads/jira/solr-9854
Commit: 5f4521de628ced22804c3fd9bcd3de5fa815c27d
Parents: c9522a3
Author: Mike McCandless <mi...@apache.org>
Authored: Fri Dec 16 18:29:40 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Fri Dec 16 18:29:40 2016 -0500

----------------------------------------------------------------------
 .../facet/src/test/org/apache/lucene/facet/TestFacetQuery.java  | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5f4521de/lucene/facet/src/test/org/apache/lucene/facet/TestFacetQuery.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/TestFacetQuery.java b/lucene/facet/src/test/org/apache/lucene/facet/TestFacetQuery.java
index f3aa079..07edd6e 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/TestFacetQuery.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/TestFacetQuery.java
@@ -81,6 +81,11 @@ public class TestFacetQuery extends FacetTestCase {
   @AfterClass
   public static void closeTestIndex() throws IOException {
     IOUtils.close(indexReader, indexWriter, indexDirectory);
+    indexReader = null;
+    indexWriter = null;
+    indexDirectory = null;
+    searcher = null;
+    config = null;
   }
 
   @Test


[19/44] lucene-solr:jira/solr-9854: SOLR-9812: Added entry to CHANGES.txt

Posted by ab...@apache.org.
SOLR-9812: Added entry to CHANGES.txt


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

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

----------------------------------------------------------------------
 solr/CHANGES.txt | 8 ++++++--
 1 file changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5a17c1b5/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index fab3076..a0f11f1 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -167,11 +167,15 @@ New Features
 * SOLR-9844: FieldCache information fetched via the mbeans handler or seen via the UI now displays the total size used.
   The individual cache entries in the response are now formatted better as well. (Varun Thacker)
 
-* SOLR-9788: Use instrumented jetty classes provided by the dropwizard metric library. This also introduces a
-  new /admin/metrics API endpoint to return all registered metrics in JSON format. (shalin)
+* SOLR-9788: Use instrumented jetty classes provided by the dropwizard metric library. (shalin)
 
 * SOLR-9805: Use metrics-jvm library to instrument jvm internals such as GC, memory usage and others. (shalin)
 
+* SOLR-9812: Added a new /admin/metrics API to return all metrics collected by Solr via API. API supports two
+  optional parameters 'group' (all,jvm,jetty,http,node,core) and 'type' (all,counter,timer,gauge,histogram) both
+  of which are multi-valued. Example: http://localhost:8983/solr/admin/metrics?group=jvm,jetty&type=counter
+  (shalin)
+
 Optimizations
 ----------------------
 * SOLR-9704: Facet Module / JSON Facet API: Optimize blockChildren facets that have


[28/44] lucene-solr:jira/solr-9854: Merge branch 'master' into feature/metrics

Posted by ab...@apache.org.
Merge branch 'master' into feature/metrics


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

Branch: refs/heads/jira/solr-9854
Commit: 67dc7604ef7340cf94ad286f3693977aec12f26f
Parents: f56da1d 321c6f0
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Mon Dec 19 13:51:25 2016 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Mon Dec 19 13:51:25 2016 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  17 +
 .../lucene70/Lucene70DocValuesConsumer.java     | 163 +++++--
 .../lucene70/Lucene70DocValuesFormat.java       |   5 +-
 .../lucene70/Lucene70DocValuesProducer.java     | 220 +++++++--
 .../org/apache/lucene/index/LeafReader.java     |   2 +-
 .../apache/lucene/index/PrefixCodedTerms.java   |   4 +-
 .../org/apache/lucene/index/SegmentInfos.java   |   7 +-
 .../java/org/apache/lucene/util/LongValues.java |   9 +
 .../apache/lucene/util/packed/DirectWriter.java |   8 +-
 .../lucene70/TestLucene70DocValuesFormat.java   | 152 +++++++
 .../org/apache/lucene/facet/FacetQuery.java     |  52 +++
 .../apache/lucene/facet/MultiFacetQuery.java    |  61 +++
 .../org/apache/lucene/facet/TestFacetQuery.java | 103 +++++
 .../apache/lucene/search/DocValuesStats.java    | 409 +++++++++++++++++
 .../lucene/search/DocValuesStatsCollector.java  |  64 +++
 .../search/TestDocValuesStatsCollector.java     | 448 +++++++++++++++++++
 .../apache/lucene/document/RangeFieldQuery.java |   1 +
 .../lucene/search/DocValuesNumbersQuery.java    |  26 +-
 .../lucene/search/DocValuesTermsQuery.java      |  49 +-
 .../org/apache/lucene/search/LongHashSet.java   | 156 +++++++
 .../search/BaseRangeFieldQueryTestCase.java     |   2 +-
 .../apache/lucene/search/LongHashSetTests.java  | 100 +++++
 .../lucene/search/TestDocValuesTermsQuery.java  |   1 +
 .../org/apache/lucene/search/QueryUtils.java    |   4 -
 solr/CHANGES.txt                                |  14 +
 .../solr/collection1/conf/solrconfig.xml        |   5 -
 solr/contrib/ltr/example/solrconfig.xml         |  11 -
 .../solr/solrcloud/conf/solrconfig.xml          |  11 -
 .../OverseerAutoReplicaFailoverThread.java      |   9 +-
 .../java/org/apache/solr/core/SolrConfig.java   |   2 +-
 .../src/java/org/apache/solr/core/SolrCore.java |  92 ++--
 .../apache/solr/search/SolrFieldCacheMBean.java |   6 +-
 .../apache/solr/servlet/SolrDispatchFilter.java |  15 +
 .../apache/solr/servlet/SolrRequestParsers.java |  31 +-
 .../solr/uninverting/UninvertingReader.java     |  25 +-
 .../solr/util/SolrFileCleaningTracker.java      | 147 ++++++
 .../solr/collection1/conf/solrconfig_perf.xml   |   1 -
 .../cloud/SegmentTerminateEarlyTestState.java   |  12 +-
 .../solr/cloud/TestMiniSolrCloudCluster.java    |  50 ---
 .../apache/solr/cloud/TestSegmentSorting.java   | 133 ++++++
 .../org/apache/solr/search/TestRealTimeGet.java |   2 +-
 .../solr/search/TestSolrFieldCacheMBean.java    |   3 +-
 .../apache/solr/search/TestStressLucene.java    |   2 +-
 .../apache/solr/search/TestStressRecovery.java  |   2 +-
 .../apache/solr/search/TestStressReorder.java   |   5 +-
 .../solr/search/TestStressUserVersions.java     |   4 +-
 .../apache/solr/search/TestStressVersions.java  |   2 +-
 .../example-DIH/solr/db/conf/solrconfig.xml     |  11 -
 .../example-DIH/solr/mail/conf/solrconfig.xml   |  11 -
 .../example-DIH/solr/rss/conf/solrconfig.xml    |  11 -
 .../example-DIH/solr/solr/conf/solrconfig.xml   |  11 -
 .../example-DIH/solr/tika/conf/solrconfig.xml   |  11 -
 solr/example/files/conf/solrconfig.xml          |  11 -
 .../basic_configs/conf/solrconfig.xml           |  11 -
 .../conf/solrconfig.xml                         |  11 -
 .../conf/solrconfig.xml                         |  11 -
 .../solr/client/solrj/LargeVolumeTestBase.java  |   3 +-
 .../solr/client/solrj/SolrExampleTestBase.java  |   1 -
 .../java/org/apache/solr/SolrJettyTestBase.java |   2 -
 .../java/org/apache/solr/SolrTestCaseJ4.java    |   2 +-
 60 files changed, 2371 insertions(+), 383 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/67dc7604/solr/CHANGES.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/67dc7604/solr/core/src/java/org/apache/solr/core/SolrCore.java
----------------------------------------------------------------------
diff --cc solr/core/src/java/org/apache/solr/core/SolrCore.java
index cad6423,b3facef..4a63686
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@@ -206,12 -200,6 +206,12 @@@ public final class SolrCore implements 
    private final ReentrantLock ruleExpiryLock;
    private final ReentrantLock snapshotDelLock; // A lock instance to guard against concurrent deletions.
  
 +  private final Timer newSearcherTimer;
 +  private final Timer newSearcherWarmupTimer;
 +  private final Counter newSearcherCounter;
-   private final Counter newSearcherMaxErrorsCounter;
++  private final Counter newSearcherMaxReachedCounter;
 +  private final Counter newSearcherOtherErrorsCounter;
 +
    public Date getStartTimeStamp() { return startTime; }
  
    private final Map<Object, IndexFingerprint> perSegmentFingerprintCache = new MapMaker().weakKeys().makeMap();
@@@ -863,16 -838,6 +863,16 @@@
  
      checkVersionFieldExistsInSchema(schema, coreDescriptor);
  
 +    // Initialize the metrics manager
 +    this.metricManager = initMetricManager(config);
 +
 +    // initialize searcher-related metrics
 +    newSearcherCounter = SolrMetricManager.counter(metricManager.getRegistryName(), "newSearcher");
 +    newSearcherTimer = SolrMetricManager.timer(metricManager.getRegistryName(), "newSearcherTime");
 +    newSearcherWarmupTimer = SolrMetricManager.timer(metricManager.getRegistryName(), "newSearcherWarmup");
-     newSearcherMaxErrorsCounter = SolrMetricManager.counter(metricManager.getRegistryName(), "newSearcherMaxErrors");
++    newSearcherMaxReachedCounter = SolrMetricManager.counter(metricManager.getRegistryName(), "newSearcherMaxReached");
 +    newSearcherOtherErrorsCounter = SolrMetricManager.counter(metricManager.getRegistryName(), "newSearcherErrors");
 +
      // Initialize JMX
      this.infoRegistry = initInfoRegistry(name, config);
      infoRegistry.put("fieldCache", new SolrFieldCacheMBean());
@@@ -1943,53 -1886,57 +1943,59 @@@
      // if it isn't necessary.
  
      synchronized (searcherLock) {
-       // see if we can return the current searcher
-       if (_searcher!=null && !forceNew) {
-         if (returnSearcher) {
-           _searcher.incref();
-           return _searcher;
-         } else {
-           return null;
+       for(;;) { // this loop is so w can retry in the event that we exceed maxWarmingSearchers
+         // see if we can return the current searcher
+         if (_searcher != null && !forceNew) {
+           if (returnSearcher) {
+             _searcher.incref();
+             return _searcher;
+           } else {
+             return null;
+           }
          }
-       }
  
-       // check to see if we can wait for someone else's searcher to be set
-       if (onDeckSearchers>0 && !forceNew && _searcher==null) {
-         try {
-           searcherLock.wait();
-         } catch (InterruptedException e) {
-           log.info(SolrException.toStr(e));
+         // check to see if we can wait for someone else's searcher to be set
+         if (onDeckSearchers > 0 && !forceNew && _searcher == null) {
+           try {
+             searcherLock.wait();
+           } catch (InterruptedException e) {
+             log.info(SolrException.toStr(e));
+           }
          }
-       }
  
-       // check again: see if we can return right now
-       if (_searcher!=null && !forceNew) {
-         if (returnSearcher) {
-           _searcher.incref();
-           return _searcher;
-         } else {
-           return null;
+         // check again: see if we can return right now
+         if (_searcher != null && !forceNew) {
+           if (returnSearcher) {
+             _searcher.incref();
+             return _searcher;
+           } else {
+             return null;
+           }
+         }
+ 
+         // At this point, we know we need to open a new searcher...
+         // first: increment count to signal other threads that we are
+         //        opening a new searcher.
+         onDeckSearchers++;
++        newSearcherCounter.inc();
+         if (onDeckSearchers < 1) {
+           // should never happen... just a sanity check
+           log.error(logid + "ERROR!!! onDeckSearchers is " + onDeckSearchers);
+           onDeckSearchers = 1;  // reset
+         } else if (onDeckSearchers > maxWarmingSearchers) {
+           onDeckSearchers--;
++          newSearcherMaxReachedCounter.inc();
+           try {
+             searcherLock.wait();
+           } catch (InterruptedException e) {
+             log.info(SolrException.toStr(e));
+           }
+           continue;  // go back to the top of the loop and retry
+         } else if (onDeckSearchers > 1) {
+           log.warn(logid + "PERFORMANCE WARNING: Overlapping onDeckSearchers=" + onDeckSearchers);
          }
-       }
  
-       // At this point, we know we need to open a new searcher...
-       // first: increment count to signal other threads that we are
-       //        opening a new searcher.
-       onDeckSearchers++;
-       newSearcherCounter.inc();
-       if (onDeckSearchers < 1) {
-         // should never happen... just a sanity check
-         log.error(logid+"ERROR!!! onDeckSearchers is " + onDeckSearchers);
-         onDeckSearchers=1;  // reset
-       } else if (onDeckSearchers > maxWarmingSearchers) {
-         onDeckSearchers--;
-         String msg="Error opening new searcher. exceeded limit of maxWarmingSearchers="+maxWarmingSearchers + ", try again later.";
-         log.warn(logid+""+ msg);
-         // HTTP 503==service unavailable, or 409==Conflict
-         newSearcherMaxErrorsCounter.inc();
-         throw new SolrException(SolrException.ErrorCode.SERVICE_UNAVAILABLE,msg);
-       } else if (onDeckSearchers > 1) {
-         log.warn(logid+"PERFORMANCE WARNING: Overlapping onDeckSearchers=" + onDeckSearchers);
+         break; // I can now exit the loop and proceed to open a searcher
        }
      }
  

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/67dc7604/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
----------------------------------------------------------------------
diff --cc solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
index cfd90cb,e8c4657..2852041
--- a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
@@@ -47,12 -45,7 +47,13 @@@ import java.util.concurrent.atomic.Atom
  import java.util.regex.Matcher;
  import java.util.regex.Pattern;
  
 +import com.codahale.metrics.jvm.BufferPoolMetricSet;
 +import com.codahale.metrics.jvm.ClassLoadingGaugeSet;
 +import com.codahale.metrics.jvm.FileDescriptorRatioGauge;
 +import com.codahale.metrics.jvm.GarbageCollectorMetricSet;
 +import com.codahale.metrics.jvm.MemoryUsageGaugeSet;
 +import com.codahale.metrics.jvm.ThreadStatesGaugeSet;
+ import org.apache.commons.io.FileCleaningTracker;
  import org.apache.commons.io.input.CloseShieldInputStream;
  import org.apache.commons.io.output.CloseShieldOutputStream;
  import org.apache.commons.lang.StringUtils;


[04/44] lucene-solr:jira/solr-9854: LUCENE-7590: fix typo in method parameter

Posted by ab...@apache.org.
LUCENE-7590: fix typo in method parameter


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

Branch: refs/heads/jira/solr-9854
Commit: 85582dabe4372085e1af5d01ebbfcfd0303b9f12
Parents: 770f1eb
Author: Shai Erera <sh...@apache.org>
Authored: Wed Dec 14 13:28:02 2016 +0200
Committer: Shai Erera <sh...@apache.org>
Committed: Wed Dec 14 13:28:02 2016 +0200

----------------------------------------------------------------------
 .../misc/src/java/org/apache/lucene/search/DocValuesStats.java | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/85582dab/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java b/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
index fad9f97..38158cf 100644
--- a/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
+++ b/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
@@ -51,7 +51,7 @@ public abstract class DocValuesStats<T> {
    * Initializes this object with the given reader context. Returns whether stats can be computed for this segment (i.e.
    * it does have the requested DocValues field).
    */
-  protected abstract boolean init(LeafReaderContext contxt) throws IOException;
+  protected abstract boolean init(LeafReaderContext context) throws IOException;
 
   /** Returns whether the given document has a value for the requested DocValues field. */
   protected abstract boolean hasValue(int doc) throws IOException;
@@ -106,8 +106,8 @@ public abstract class DocValuesStats<T> {
     }
 
     @Override
-    protected final boolean init(LeafReaderContext contxt) throws IOException {
-      ndv = contxt.reader().getNumericDocValues(field);
+    protected final boolean init(LeafReaderContext context) throws IOException {
+      ndv = context.reader().getNumericDocValues(field);
       return ndv != null;
     }
 


[37/44] lucene-solr:jira/solr-9854: SOLR-4735 Un-staticize SolrMetricManager and other fixes: * static methods and SharedMetricsRegistries caused issues when multiple CoreContainers were present under the same classloader. * However, use SharedMetricsReg

Posted by ab...@apache.org.
SOLR-4735 Un-staticize SolrMetricManager and other fixes:
* static methods and SharedMetricsRegistries caused issues
when multiple CoreContainers were present under the same classloader.
* However, use SharedMetricsRegistries for solr.jetty and solr.jvm registries.
* fix JVM metrics registration to use a prefix.


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

Branch: refs/heads/jira/solr-9854
Commit: adfb4a5052afc0885905702d4858a303b2f710ae
Parents: 67dc760
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Mon Dec 19 21:29:29 2016 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Mon Dec 19 21:29:29 2016 +0100

----------------------------------------------------------------------
 .../org/apache/solr/core/CoreContainer.java     |  29 +++--
 .../src/java/org/apache/solr/core/SolrCore.java |  36 +++---
 .../apache/solr/handler/RequestHandlerBase.java |  14 +--
 .../solr/handler/admin/MetricsHandler.java      |   9 +-
 .../solr/metrics/SolrCoreMetricManager.java     |  14 ++-
 .../apache/solr/metrics/SolrMetricManager.java  | 114 +++++++++++++------
 .../apache/solr/metrics/SolrMetricProducer.java |   4 +-
 .../apache/solr/metrics/SolrMetricReporter.java |   5 +-
 .../solr/metrics/reporters/SolrJmxReporter.java |   6 +-
 .../apache/solr/servlet/SolrDispatchFilter.java |  16 +--
 .../solr/metrics/SolrCoreMetricManagerTest.java |  42 +++----
 .../solr/metrics/SolrMetricManagerTest.java     |  83 ++++++++------
 .../solr/metrics/SolrMetricReporterTest.java    |   4 +-
 .../solr/metrics/SolrMetricTestUtils.java       |   6 +-
 .../metrics/SolrMetricsIntegrationTest.java     |  16 +--
 .../metrics/reporters/MockMetricReporter.java   |   6 +-
 .../metrics/reporters/SolrJmxReporterTest.java  |  26 +++--
 .../java/org/apache/solr/util/TestHarness.java  |  10 +-
 18 files changed, 260 insertions(+), 180 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/adfb4a50/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 0703212..6e640bc 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -161,6 +161,8 @@ public class CoreContainer {
 
   private BackupRepositoryFactory backupRepoFactory;
 
+  protected SolrMetricManager metricManager;
+
   protected MetricsHandler metricsHandler;
 
   /**
@@ -430,6 +432,10 @@ public class CoreContainer {
     return pkiAuthenticationPlugin;
   }
 
+  public SolrMetricManager getMetricManager() {
+    return metricManager;
+  }
+
   //-------------------------------------------------------------------
   // Initialization / Cleanup
   //-------------------------------------------------------------------
@@ -470,6 +476,8 @@ public class CoreContainer {
 
     MDCLoggingContext.setNode(this);
 
+    metricManager = new SolrMetricManager();
+
     securityConfHandler = isZooKeeperAware() ? new SecurityConfHandlerZk(this) : new SecurityConfHandlerLocal(this);
     reloadSecurityProperties();
     this.backupRepoFactory = new BackupRepositoryFactory(cfg.getBackupRepositoryPlugins());
@@ -481,12 +489,15 @@ public class CoreContainer {
     configSetsHandler = createHandler(CONFIGSETS_HANDLER_PATH, cfg.getConfigSetsHandlerClass(), ConfigSetsHandler.class);
     metricsHandler = createHandler(METRICS_PATH, MetricsHandler.class.getName(), MetricsHandler.class);
     containerHandlers.put(AUTHZ_PATH, securityConfHandler);
-    securityConfHandler.initializeMetrics(SolrInfoMBean.Group.node.toString(), AUTHZ_PATH);
+    securityConfHandler.initializeMetrics(metricManager, SolrInfoMBean.Group.node.toString(), AUTHZ_PATH);
     containerHandlers.put(AUTHC_PATH, securityConfHandler);
     if(pkiAuthenticationPlugin != null)
       containerHandlers.put(PKIAuthenticationPlugin.PATH, pkiAuthenticationPlugin.getRequestHandler());
 
-    SolrMetricManager.loadReporters(cfg.getMetricReporterPlugins(), loader, SolrInfoMBean.Group.node);
+    metricManager.loadReporters(cfg.getMetricReporterPlugins(), loader, SolrInfoMBean.Group.node);
+    metricManager.loadReporters(cfg.getMetricReporterPlugins(), loader, SolrInfoMBean.Group.jvm);
+    metricManager.loadReporters(cfg.getMetricReporterPlugins(), loader, SolrInfoMBean.Group.jetty);
+    metricManager.loadReporters(cfg.getMetricReporterPlugins(), loader, SolrInfoMBean.Group.http);
 
     coreConfigService = ConfigSetService.createConfigSetService(cfg, loader, zkSys.zkController);
 
@@ -497,11 +508,11 @@ public class CoreContainer {
     Gauge<Integer> lazyCores = () -> solrCores.getCoreNames().size() - solrCores.getCores().size();
     Gauge<Integer> unloadedCores = () -> solrCores.getAllCoreNames().size() - solrCores.getCoreNames().size();
 
-    SolrMetricManager.register(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node),
+    metricManager.register(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node),
         loadedCores, true, "loaded", "cores");
-    SolrMetricManager.register(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node),
+    metricManager.register(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node),
         lazyCores, true, "lazy", "cores");
-    SolrMetricManager.register(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node),
+    metricManager.register(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node),
         unloadedCores, true, "unloaded", "cores");
 
     // setup executor to load cores in parallel
@@ -677,7 +688,9 @@ public class CoreContainer {
       }
     }
 
-    SolrMetricManager.closeReporters(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node));
+    if (metricManager != null) {
+      metricManager.closeReporters(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node));
+    }
 
     // It should be safe to close the authorization plugin at this point.
     try {
@@ -1056,7 +1069,7 @@ public class CoreContainer {
     coresLocator.delete(this, cd);
 
     // delete metrics specific to this core
-    SolrMetricManager.removeRegistry(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.core, name));
+    metricManager.removeRegistry(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.core, name));
 
     if (core == null) {
       // transient core
@@ -1197,7 +1210,7 @@ public class CoreContainer {
       containerHandlers.put(path, (SolrRequestHandler)handler);
     }
     if (handler instanceof SolrMetricProducer) {
-      ((SolrMetricProducer)handler).initializeMetrics(SolrInfoMBean.Group.node.toString(), path);
+      ((SolrMetricProducer)handler).initializeMetrics(metricManager, SolrInfoMBean.Group.node.toString(), path);
     }
     return handler;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/adfb4a50/solr/core/src/java/org/apache/solr/core/SolrCore.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrCore.java b/solr/core/src/java/org/apache/solr/core/SolrCore.java
index 4a63686..3d78c40 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -192,7 +192,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
   private final PluginBag<SearchComponent> searchComponents = new PluginBag<>(SearchComponent.class, this);
   private final PluginBag<UpdateRequestProcessorFactory> updateProcessors = new PluginBag<>(UpdateRequestProcessorFactory.class, this, true);
   private final Map<String,UpdateRequestProcessorChain> updateProcessorChains;
-  private final SolrCoreMetricManager metricManager;
+  private final SolrCoreMetricManager coreMetricManager;
   private final Map<String, SolrInfoMBean> infoRegistry;
   private final IndexDeletionPolicyWrapper solrDelPolicy;
   private final SolrSnapshotMetaDataManager snapshotMgr;
@@ -402,8 +402,8 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
     this.name = v;
     this.logid = (v==null)?"":("["+v+"] ");
     this.coreDescriptor = new CoreDescriptor(v, this.coreDescriptor);
-    if (metricManager != null) {
-      metricManager.afterCoreSetName();
+    if (coreMetricManager != null) {
+      coreMetricManager.afterCoreSetName();
     }
   }
 
@@ -417,8 +417,8 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
    *
    * @return the {@link SolrCoreMetricManager} for this core
    */
-  public SolrCoreMetricManager getMetricManager() {
-    return metricManager;
+  public SolrCoreMetricManager getCoreMetricManager() {
+    return coreMetricManager;
   }
 
   /**
@@ -864,14 +864,16 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
     checkVersionFieldExistsInSchema(schema, coreDescriptor);
 
     // Initialize the metrics manager
-    this.metricManager = initMetricManager(config);
+    this.coreMetricManager = initCoreMetricManager(config);
+
+    SolrMetricManager metricManager = this.coreDescriptor.getCoreContainer().getMetricManager();
 
     // initialize searcher-related metrics
-    newSearcherCounter = SolrMetricManager.counter(metricManager.getRegistryName(), "newSearcher");
-    newSearcherTimer = SolrMetricManager.timer(metricManager.getRegistryName(), "newSearcherTime");
-    newSearcherWarmupTimer = SolrMetricManager.timer(metricManager.getRegistryName(), "newSearcherWarmup");
-    newSearcherMaxReachedCounter = SolrMetricManager.counter(metricManager.getRegistryName(), "newSearcherMaxReached");
-    newSearcherOtherErrorsCounter = SolrMetricManager.counter(metricManager.getRegistryName(), "newSearcherErrors");
+    newSearcherCounter = metricManager.counter(coreMetricManager.getRegistryName(), "newSearcher");
+    newSearcherTimer = metricManager.timer(coreMetricManager.getRegistryName(), "newSearcherTime");
+    newSearcherWarmupTimer = metricManager.timer(coreMetricManager.getRegistryName(), "newSearcherWarmup");
+    newSearcherMaxReachedCounter = metricManager.counter(coreMetricManager.getRegistryName(), "newSearcherMaxReached");
+    newSearcherOtherErrorsCounter = metricManager.counter(coreMetricManager.getRegistryName(), "newSearcherErrors");
 
     // Initialize JMX
     this.infoRegistry = initInfoRegistry(name, config);
@@ -1083,10 +1085,10 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
    * @param config the given configuration
    * @return an instance of {@link SolrCoreMetricManager}
    */
-  private SolrCoreMetricManager initMetricManager(SolrConfig config) {
-    SolrCoreMetricManager metricManager = new SolrCoreMetricManager(this);
-    metricManager.loadReporters();
-    return metricManager;
+  private SolrCoreMetricManager initCoreMetricManager(SolrConfig config) {
+    SolrCoreMetricManager coreMetricManager = new SolrCoreMetricManager(this);
+    coreMetricManager.loadReporters();
+    return coreMetricManager;
   }
 
   private Map<String,SolrInfoMBean> initInfoRegistry(String name, SolrConfig config) {
@@ -1410,7 +1412,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
     }
 
     try {
-      metricManager.close();
+      coreMetricManager.close();
     } catch (Throwable e) {
       SolrException.log(log, e);
       if (e instanceof  Error) {
@@ -2819,7 +2821,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
 
     if (solrInfoMBean instanceof SolrMetricProducer) {
       SolrMetricProducer producer = (SolrMetricProducer) solrInfoMBean;
-      metricManager.registerMetricProducer(name, producer);
+      coreMetricManager.registerMetricProducer(name, producer);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/adfb4a50/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java b/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
index 7d9d464..85597dc 100644
--- a/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
+++ b/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
@@ -133,13 +133,13 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
   }
 
   @Override
-  public Collection<String> initializeMetrics(String registryName, String scope) {
-    numErrors = SolrMetricManager.meter(registryName, "errors", getCategory().toString(), scope);
-    numServerErrors = SolrMetricManager.meter(registryName, "serverErrors", getCategory().toString(), scope);
-    numClientErrors = SolrMetricManager.meter(registryName, "clientErrors", getCategory().toString(), scope);
-    numTimeouts = SolrMetricManager.meter(registryName, "timeouts", getCategory().toString(), scope);
-    requests = SolrMetricManager.counter(registryName, "requests", getCategory().toString(), scope);
-    requestTimes = SolrMetricManager.timer(registryName, "requestTimes", getCategory().toString(), scope);
+  public Collection<String> initializeMetrics(SolrMetricManager manager, String registryName, String scope) {
+    numErrors = manager.meter(registryName, "errors", getCategory().toString(), scope);
+    numServerErrors = manager.meter(registryName, "serverErrors", getCategory().toString(), scope);
+    numClientErrors = manager.meter(registryName, "clientErrors", getCategory().toString(), scope);
+    numTimeouts = manager.meter(registryName, "timeouts", getCategory().toString(), scope);
+    requests = manager.counter(registryName, "requests", getCategory().toString(), scope);
+    requestTimes = manager.timer(registryName, "requestTimes", getCategory().toString(), scope);
     return Arrays.asList("errors", "serverErrors", "clientErrors", "timeouts", "requestTimes", "requests");
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/adfb4a50/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 1adc480..78b2045 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
@@ -48,13 +48,16 @@ import org.apache.solr.util.stats.MetricUtils;
  */
 public class MetricsHandler extends RequestHandlerBase implements PermissionNameProvider {
   final CoreContainer container;
+  final SolrMetricManager metricManager;
 
   public MetricsHandler() {
     this.container = null;
+    this.metricManager = null;
   }
 
   public MetricsHandler(CoreContainer container) {
     this.container = container;
+    this.metricManager = this.container.getMetricManager();
   }
 
   @Override
@@ -80,13 +83,13 @@ public class MetricsHandler extends RequestHandlerBase implements PermissionName
         container.getAllCoreNames().forEach(s -> {
           String coreRegistryName;
           try (SolrCore core = container.getCore(s)) {
-            coreRegistryName = core.getMetricManager().getRegistryName();
+            coreRegistryName = core.getCoreMetricManager().getRegistryName();
           }
-          MetricRegistry registry = SolrMetricManager.registry(coreRegistryName);
+          MetricRegistry registry = metricManager.registry(coreRegistryName);
           response.add(coreRegistryName, MetricUtils.toNamedList(registry, metricFilters));
         });
       } else {
-        MetricRegistry registry = SolrMetricManager.registry(registryName);
+        MetricRegistry registry = metricManager.registry(registryName);
         response.add(registryName, MetricUtils.toNamedList(registry, metricFilters));
       }
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/adfb4a50/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java b/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java
index 18307ee..0e5403b 100644
--- a/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java
@@ -37,6 +37,7 @@ public class SolrCoreMetricManager implements Closeable {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   private final SolrCore core;
+  private final SolrMetricManager metricManager;
   private String registryName;
 
   /**
@@ -46,6 +47,7 @@ public class SolrCoreMetricManager implements Closeable {
    */
   public SolrCoreMetricManager(SolrCore core) {
     this.core = core;
+    this.metricManager = core.getCoreDescriptor().getCoreContainer().getMetricManager();
     registryName = createRegistryName(core.getCoreDescriptor().getCollectionName(), core.getName());
   }
 
@@ -56,7 +58,7 @@ public class SolrCoreMetricManager implements Closeable {
   public void loadReporters() {
     NodeConfig nodeConfig = core.getCoreDescriptor().getCoreContainer().getConfig();
     PluginInfo[] pluginInfos = nodeConfig.getMetricReporterPlugins();
-    SolrMetricManager.loadReporters(pluginInfos, core.getResourceLoader(), SolrInfoMBean.Group.core, registryName);
+    metricManager.loadReporters(pluginInfos, core.getResourceLoader(), SolrInfoMBean.Group.core, registryName);
   }
 
   /**
@@ -71,10 +73,10 @@ public class SolrCoreMetricManager implements Closeable {
       return;
     }
     // close old reporters
-    SolrMetricManager.closeReporters(oldRegistryName);
-    SolrMetricManager.moveMetrics(oldRegistryName, registryName, null);
+    metricManager.closeReporters(oldRegistryName);
+    metricManager.moveMetrics(oldRegistryName, registryName, null);
     // old registry is no longer used - we have moved the metrics
-    SolrMetricManager.removeRegistry(oldRegistryName);
+    metricManager.removeRegistry(oldRegistryName);
     // load reporters again, using the new core name
     loadReporters();
   }
@@ -90,7 +92,7 @@ public class SolrCoreMetricManager implements Closeable {
       throw new IllegalArgumentException("registerMetricProducer() called with illegal arguments: " +
           "scope = " + scope + ", producer = " + producer);
     }
-    Collection<String> registered = producer.initializeMetrics(getRegistryName(), scope);
+    Collection<String> registered = producer.initializeMetrics(metricManager, getRegistryName(), scope);
     if (registered == null || registered.isEmpty()) {
       throw new IllegalArgumentException("registerMetricProducer() did not register any metrics " +
       "for scope = " + scope + ", producer = " + producer);
@@ -102,7 +104,7 @@ public class SolrCoreMetricManager implements Closeable {
    */
   @Override
   public void close() throws IOException {
-    SolrMetricManager.closeReporters(getRegistryName());
+    metricManager.closeReporters(getRegistryName());
   }
 
   public SolrCore getCore() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/adfb4a50/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java b/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
index b700782..3f51aef 100644
--- a/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
@@ -24,6 +24,8 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
@@ -59,7 +61,9 @@ import org.slf4j.LoggerFactory;
  * names.</p>
  * <p>Solr uses several different registries for collecting metrics belonging to different groups, using
  * {@link org.apache.solr.core.SolrInfoMBean.Group} as the main name of the registry (plus the
- * above-mentioned prefix).</p>
+ * above-mentioned prefix). Instances of {@link SolrMetricManager} are created for each {@link org.apache.solr.core.CoreContainer},
+ * and most registries are local to each instance, with the exception of two global registries:
+ * <code>solr.jetty</code> and <code>solr.jvm</code>, which are shared between all {@link org.apache.solr.core.CoreContainer}-s</p>
  */
 public class SolrMetricManager {
 
@@ -67,12 +71,21 @@ public class SolrMetricManager {
 
   public static final String REGISTRY_NAME_PREFIX = "solr.";
 
-  private static final Map<String, Map<String, SolrMetricReporter>> reporters = new HashMap<>();
+  public static final String JETTY_REGISTRY = REGISTRY_NAME_PREFIX + SolrInfoMBean.Group.jetty.toString();
 
-  private static final Lock reportersLock = new ReentrantLock();
+  public static final String JVM_REGISTRY = REGISTRY_NAME_PREFIX + SolrInfoMBean.Group.jvm.toString();
 
-  // don't create instances of this class
-  private SolrMetricManager() { }
+  private final ConcurrentMap<String, MetricRegistry> registries = new ConcurrentHashMap<>();
+
+  // these reporters are per CoreContainer
+  private final Map<String, Map<String, SolrMetricReporter>> reporters = new HashMap<>();
+
+  // these reporters are per JVM
+  private static final Map<String, Map<String, SolrMetricReporter>> sharedReporters = new HashMap<>();
+
+  private final Lock reportersLock = new ReentrantLock();
+
+  public SolrMetricManager() { }
 
   /**
    * An implementation of {@link MetricFilter} that selects metrics
@@ -125,8 +138,11 @@ public class SolrMetricManager {
   /**
    * Return a set of existing registry names.
    */
-  public static Set<String> registryNames() {
-    return SharedMetricRegistries.names();
+  public Set<String> registryNames() {
+    Set<String> set = new HashSet<>();
+    set.addAll(registries.keySet());
+    set.addAll(SharedMetricRegistries.names());
+    return Collections.unmodifiableSet(set);
   }
 
   /**
@@ -134,20 +150,40 @@ public class SolrMetricManager {
    * @param registry name of the registry
    * @return existing or newly created registry
    */
-  public static MetricRegistry registry(String registry) {
-    return SharedMetricRegistries.getOrCreate(overridableRegistryName(registry));
+  public MetricRegistry registry(String registry) {
+    registry = overridableRegistryName(registry);
+    if (JETTY_REGISTRY.equals(registry) || JVM_REGISTRY.equals(registry)) {
+      return SharedMetricRegistries.getOrCreate(registry);
+    } else {
+      final MetricRegistry existing = registries.get(registry);
+      if (existing == null) {
+        final MetricRegistry created = new MetricRegistry();
+        final MetricRegistry raced = registries.putIfAbsent(registry, created);
+        if (raced == null) {
+          return created;
+        } else {
+          return raced;
+        }
+      } else {
+        return existing;
+      }
+    }
   }
 
   /**
    * Remove a named registry.
    * @param registry name of the registry to remove
    */
-  public static void removeRegistry(String registry) {
+  public void removeRegistry(String registry) {
     // close any reporters for this registry first
     closeReporters(registry);
     // make sure we use a name with prefix, with overrides
     registry = overridableRegistryName(registry);
-    SharedMetricRegistries.remove(registry);
+    if (JETTY_REGISTRY.equals(registry) || JVM_REGISTRY.equals(registry)) {
+      SharedMetricRegistries.remove(registry);
+    } else {
+      registries.remove(registry);
+    }
   }
 
   /**
@@ -158,7 +194,7 @@ public class SolrMetricManager {
    * @param filter optional {@link MetricFilter} to select what metrics to move. If null
    *               then all metrics will be moved.
    */
-  public static void moveMetrics(String fromRegistry, String toRegistry, MetricFilter filter) {
+  public void moveMetrics(String fromRegistry, String toRegistry, MetricFilter filter) {
     MetricRegistry from = registry(fromRegistry);
     MetricRegistry to = registry(toRegistry);
     if (from == to) {
@@ -182,19 +218,23 @@ public class SolrMetricManager {
    * already exist.
    * @param registry registry name
    * @param metrics metric set to register
-   * @param skipExisting if true then already existing metrics with the same name will be kept.
+   * @param force if true then already existing metrics with the same name will be replaced.
    *                     When false and a metric with the same name already exists an exception
    *                     will be thrown.
+   * @param metricPath (optional) additional top-most metric name path elements
    * @throws Exception if a metric with this name already exists.
    */
-  public static void registerAll(String registry, MetricSet metrics, boolean skipExisting) throws Exception {
+  public void registerAll(String registry, MetricSet metrics, boolean force, String... metricPath) throws Exception {
     MetricRegistry metricRegistry = registry(registry);
-    Map<String, Metric> existingMetrics = metricRegistry.getMetrics();
-    for (Map.Entry<String, Metric> entry : metrics.getMetrics().entrySet()) {
-      if (skipExisting && existingMetrics.containsKey(entry.getKey())) {
-        continue;
+    synchronized (metricRegistry) {
+      Map<String, Metric> existingMetrics = metricRegistry.getMetrics();
+      for (Map.Entry<String, Metric> entry : metrics.getMetrics().entrySet()) {
+        String fullName = mkName(entry.getKey(), metricPath);
+        if (force && existingMetrics.containsKey(fullName)) {
+          metricRegistry.remove(fullName);
+        }
+        metricRegistry.register(fullName, entry.getValue());
       }
-      metricRegistry.register(entry.getKey(), entry.getValue());
     }
   }
 
@@ -202,7 +242,7 @@ public class SolrMetricManager {
    * Remove all metrics from a specified registry.
    * @param registry registry name
    */
-  public static void clearRegistry(String registry) {
+  public void clearRegistry(String registry) {
     registry(registry).removeMatching(MetricFilter.ALL);
   }
 
@@ -216,7 +256,7 @@ public class SolrMetricManager {
    *        with the prefix will be removed.
    * @return set of metrics names that have been removed.
    */
-  public static Set<String> clearMetrics(String registry, String... metricPath) {
+  public Set<String> clearMetrics(String registry, String... metricPath) {
     PrefixFilter filter;
     if (metricPath == null || metricPath.length == 0) {
       filter = new PrefixFilter("");
@@ -236,7 +276,7 @@ public class SolrMetricManager {
    * @param metricPath (optional) additional top-most metric name path elements
    * @return existing or a newly created {@link Meter}
    */
-  public static Meter meter(String registry, String metricName, String... metricPath) {
+  public Meter meter(String registry, String metricName, String... metricPath) {
     return registry(registry).meter(mkName(metricName, metricPath));
   }
 
@@ -248,7 +288,7 @@ public class SolrMetricManager {
    * @param metricPath (optional) additional top-most metric name path elements
    * @return existing or a newly created {@link Timer}
    */
-  public static Timer timer(String registry, String metricName, String... metricPath) {
+  public Timer timer(String registry, String metricName, String... metricPath) {
     return registry(registry).timer(mkName(metricName, metricPath));
   }
 
@@ -260,7 +300,7 @@ public class SolrMetricManager {
    * @param metricPath (optional) additional top-most metric name path elements
    * @return existing or a newly created {@link Counter}
    */
-  public static Counter counter(String registry, String metricName, String... metricPath) {
+  public Counter counter(String registry, String metricName, String... metricPath) {
     return registry(registry).counter(mkName(metricName, metricPath));
   }
 
@@ -272,7 +312,7 @@ public class SolrMetricManager {
    * @param metricPath (optional) additional top-most metric name path elements
    * @return existing or a newly created {@link Histogram}
    */
-  public static Histogram histogram(String registry, String metricName, String... metricPath) {
+  public Histogram histogram(String registry, String metricName, String... metricPath) {
     return registry(registry).histogram(mkName(metricName, metricPath));
   }
 
@@ -287,13 +327,15 @@ public class SolrMetricManager {
    *                   using dotted notation
    * @param metricPath (optional) additional top-most metric name path elements
    */
-  public static void register(String registry, Metric metric, boolean force, String metricName, String... metricPath) {
+  public void register(String registry, Metric metric, boolean force, String metricName, String... metricPath) {
     MetricRegistry metricRegistry = registry(registry);
     String fullName = mkName(metricName, metricPath);
-    if (force && metricRegistry.getMetrics().containsKey(fullName)) {
-      metricRegistry.remove(fullName);
+    synchronized (metricRegistry) {
+      if (force && metricRegistry.getMetrics().containsKey(fullName)) {
+        metricRegistry.remove(fullName);
+      }
+      metricRegistry.register(fullName, metric);
     }
-    metricRegistry.register(fullName, metric);
   }
 
 
@@ -405,7 +447,7 @@ public class SolrMetricManager {
    * @param group selected group, not null
    * @param registryNames optional child registry name elements
    */
-  public static void loadReporters(PluginInfo[] pluginInfos, SolrResourceLoader loader, SolrInfoMBean.Group group, String... registryNames) {
+  public void loadReporters(PluginInfo[] pluginInfos, SolrResourceLoader loader, SolrInfoMBean.Group group, String... registryNames) {
     if (pluginInfos == null || pluginInfos.length == 0) {
       return;
     }
@@ -459,7 +501,7 @@ public class SolrMetricManager {
    * @param pluginInfo plugin configuration. Plugin "name" and "class" attributes are required.
    * @throws Exception if any argument is missing or invalid
    */
-  public static void loadReporter(String registry, SolrResourceLoader loader, PluginInfo pluginInfo) throws Exception {
+  public void loadReporter(String registry, SolrResourceLoader loader, PluginInfo pluginInfo) throws Exception {
     if (registry == null || pluginInfo == null || pluginInfo.name == null || pluginInfo.className == null) {
       throw new IllegalArgumentException("loadReporter called with missing arguments: " +
           "registry=" + registry + ", loader=" + loader + ", pluginInfo=" + pluginInfo);
@@ -470,8 +512,8 @@ public class SolrMetricManager {
         pluginInfo.className,
         SolrMetricReporter.class,
         new String[0],
-        new Class[] { String.class },
-        new Object[] { registry }
+        new Class[] { SolrMetricManager.class, String.class },
+        new Object[] { this, registry }
     );
     try {
       reporter.init(pluginInfo);
@@ -509,7 +551,7 @@ public class SolrMetricManager {
    * @param name reporter name
    * @return true if a named reporter existed and was closed.
    */
-  public static boolean closeReporter(String registry, String name) {
+  public boolean closeReporter(String registry, String name) {
     // make sure we use a name with prefix, with overrides
     registry = overridableRegistryName(registry);
     try {
@@ -546,7 +588,7 @@ public class SolrMetricManager {
    * @param registry registry name
    * @return names of closed reporters
    */
-  public static Set<String> closeReporters(String registry) {
+  public Set<String> closeReporters(String registry) {
     // make sure we use a name with prefix, with overrides
     registry = overridableRegistryName(registry);
     try {
@@ -583,7 +625,7 @@ public class SolrMetricManager {
    * @param registry registry name
    * @return map of reporters and their names, may be empty but never null
    */
-  public static Map<String, SolrMetricReporter> getReporters(String registry) {
+  public Map<String, SolrMetricReporter> getReporters(String registry) {
     // make sure we use a name with prefix, with overrides
     registry = overridableRegistryName(registry);
     try {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/adfb4a50/solr/core/src/java/org/apache/solr/metrics/SolrMetricProducer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrMetricProducer.java b/solr/core/src/java/org/apache/solr/metrics/SolrMetricProducer.java
index 929e6a2..b09a90a 100644
--- a/solr/core/src/java/org/apache/solr/metrics/SolrMetricProducer.java
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrMetricProducer.java
@@ -18,6 +18,7 @@ package org.apache.solr.metrics;
 
 import java.util.Collection;
 
+import com.codahale.metrics.MetricRegistry;
 import org.apache.solr.core.SolrInfoMBean;
 
 /**
@@ -28,10 +29,11 @@ public interface SolrMetricProducer extends SolrInfoMBean {
 
   /**
    * Initializes metrics specific to this producer
+   * @param manager an instance of {@link SolrMetricManager}
    * @param registry registry name where metrics are registered
    * @param scope scope of the metrics (eg. handler name) to separate metrics of
    *              instances of the same component executing in different contexts
    * @return registered (or existing) unqualified names of metrics specific to this producer.
    */
-  Collection<String> initializeMetrics(String registry, String scope);
+  Collection<String> initializeMetrics(SolrMetricManager manager, String registry, String scope);
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/adfb4a50/solr/core/src/java/org/apache/solr/metrics/SolrMetricReporter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrMetricReporter.java b/solr/core/src/java/org/apache/solr/metrics/SolrMetricReporter.java
index cfe5d67..a36a1c3 100644
--- a/solr/core/src/java/org/apache/solr/metrics/SolrMetricReporter.java
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrMetricReporter.java
@@ -18,6 +18,7 @@ package org.apache.solr.metrics;
 
 import java.io.Closeable;
 
+import com.codahale.metrics.MetricRegistry;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.util.SolrPluginUtils;
 import org.apache.solr.util.plugin.PluginInfoInitialized;
@@ -28,6 +29,7 @@ import org.apache.solr.util.plugin.PluginInfoInitialized;
 public abstract class SolrMetricReporter implements Closeable, PluginInfoInitialized {
 
   protected final String registryName;
+  protected final SolrMetricManager metricManager;
   protected PluginInfo pluginInfo;
 
   /**
@@ -35,8 +37,9 @@ public abstract class SolrMetricReporter implements Closeable, PluginInfoInitial
    * @param registryName registry to use, one of registries managed by
    *                     {@link SolrMetricManager}
    */
-  protected SolrMetricReporter(String registryName) {
+  protected SolrMetricReporter(SolrMetricManager metricManager, String registryName) {
     this.registryName = registryName;
+    this.metricManager = metricManager;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/adfb4a50/solr/core/src/java/org/apache/solr/metrics/reporters/SolrJmxReporter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/reporters/SolrJmxReporter.java b/solr/core/src/java/org/apache/solr/metrics/reporters/SolrJmxReporter.java
index 380bbaa..47fbf11 100644
--- a/solr/core/src/java/org/apache/solr/metrics/reporters/SolrJmxReporter.java
+++ b/solr/core/src/java/org/apache/solr/metrics/reporters/SolrJmxReporter.java
@@ -55,8 +55,8 @@ public class SolrJmxReporter extends SolrMetricReporter {
    *
    * @param registryName name of the registry to report
    */
-  public SolrJmxReporter(String registryName) {
-    super(registryName);
+  public SolrJmxReporter(SolrMetricManager metricManager, String registryName) {
+    super(metricManager, registryName);
     setDomain(registryName);
   }
 
@@ -99,7 +99,7 @@ public class SolrJmxReporter extends SolrMetricReporter {
 
     JmxObjectNameFactory jmxObjectNameFactory = new JmxObjectNameFactory(pluginInfo.name, domain);
 
-    reporter = JmxReporter.forRegistry(SolrMetricManager.registry(registryName))
+    reporter = JmxReporter.forRegistry(metricManager.registry(registryName))
                           .registerWith(mBeanServer)
                           .inDomain(domain)
                           .createsObjectNamesWith(jmxObjectNameFactory)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/adfb4a50/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
index 2852041..dbc4b35 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
@@ -156,7 +156,6 @@ public class SolrDispatchFilter extends BaseSolrFilter {
         excludePatterns.add(Pattern.compile(element));
       }
     }
-    setupJvmMetrics();
     try {
       Properties extraProperties = (Properties) config.getServletContext().getAttribute(PROPERTIES_ATTRIBUTE);
       if (extraProperties == null)
@@ -168,6 +167,7 @@ public class SolrDispatchFilter extends BaseSolrFilter {
       this.cores = createCoreContainer(solrHome == null ? SolrResourceLoader.locateSolrHome() : Paths.get(solrHome),
                                        extraProperties);
       this.httpClient = cores.getUpdateShardHandler().getHttpClient();
+      setupJvmMetrics();
       log.debug("user.dir=" + System.getProperty("user.dir"));
     }
     catch( Throwable t ) {
@@ -184,15 +184,15 @@ public class SolrDispatchFilter extends BaseSolrFilter {
 
   private void setupJvmMetrics()  {
     MBeanServer platformMBeanServer = ManagementFactory.getPlatformMBeanServer();
+    SolrMetricManager metricManager = cores.getMetricManager();
     try {
       String registry = SolrMetricManager.getRegistryName(SolrInfoMBean.Group.jvm);
-      SolrMetricManager.registerAll(registry, new BufferPoolMetricSet(platformMBeanServer), true);
-      SolrMetricManager.registerAll(registry, new BufferPoolMetricSet(platformMBeanServer), true);
-      SolrMetricManager.registerAll(registry, new ClassLoadingGaugeSet(), true);
-      SolrMetricManager.register(registry, new FileDescriptorRatioGauge(), true, "fileDescriptorRatio");
-      SolrMetricManager.registerAll(registry, new GarbageCollectorMetricSet(), true);
-      SolrMetricManager.registerAll(registry, new MemoryUsageGaugeSet(), true);
-      SolrMetricManager.registerAll(registry, new ThreadStatesGaugeSet(), true); // todo should we use CachedThreadStatesGaugeSet instead?
+      metricManager.registerAll(registry, new BufferPoolMetricSet(platformMBeanServer), true, "bufferPools");
+      metricManager.registerAll(registry, new ClassLoadingGaugeSet(), true, "classLoading");
+      metricManager.register(registry, new FileDescriptorRatioGauge(), true, "fileDescriptorRatio");
+      metricManager.registerAll(registry, new GarbageCollectorMetricSet(), true, "gc");
+      metricManager.registerAll(registry, new MemoryUsageGaugeSet(), true, "memory");
+      metricManager.registerAll(registry, new ThreadStatesGaugeSet(), true, "threads"); // todo should we use CachedThreadStatesGaugeSet instead?
     } catch (Exception e) {
       log.warn("Error registering JVM metrics", e);
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/adfb4a50/solr/core/src/test/org/apache/solr/metrics/SolrCoreMetricManagerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/SolrCoreMetricManagerTest.java b/solr/core/src/test/org/apache/solr/metrics/SolrCoreMetricManagerTest.java
index f532d54..65ffb93 100644
--- a/solr/core/src/test/org/apache/solr/metrics/SolrCoreMetricManagerTest.java
+++ b/solr/core/src/test/org/apache/solr/metrics/SolrCoreMetricManagerTest.java
@@ -39,18 +39,20 @@ import org.junit.Test;
 public class SolrCoreMetricManagerTest extends SolrTestCaseJ4 {
   private static final int MAX_ITERATIONS = 100;
 
-  private SolrCoreMetricManager metricManager;
+  private SolrCoreMetricManager coreMetricManager;
+  private SolrMetricManager metricManager;
 
   @Before
   public void beforeTest() throws Exception {
     initCore("solrconfig-basic.xml", "schema.xml");
-    metricManager = new SolrCoreMetricManager(h.getCore());
+    coreMetricManager = h.getCore().getCoreMetricManager();
+    metricManager = h.getCore().getCoreDescriptor().getCoreContainer().getMetricManager();
   }
 
   @After
   public void afterTest() throws IOException {
-    metricManager.close();
-    assertTrue(SolrMetricManager.getReporters(metricManager.getRegistryName()).isEmpty());
+    coreMetricManager.close();
+    assertTrue(metricManager.getReporters(coreMetricManager.getRegistryName()).isEmpty());
     deleteCore();
   }
 
@@ -61,17 +63,17 @@ public class SolrCoreMetricManagerTest extends SolrTestCaseJ4 {
     String scope = SolrMetricTestUtils.getRandomScope(random);
     SolrInfoMBean.Category category = SolrMetricTestUtils.getRandomCategory(random);
     Map<String, Counter> metrics = SolrMetricTestUtils.getRandomMetrics(random);
-    SolrMetricProducer producer = SolrMetricTestUtils.getProducerOf(category, scope, metrics);
+    SolrMetricProducer producer = SolrMetricTestUtils.getProducerOf(metricManager, category, scope, metrics);
     try {
-      metricManager.registerMetricProducer(scope, producer);
+      coreMetricManager.registerMetricProducer(scope, producer);
       assertNotNull(scope);
       assertNotNull(category);
       assertNotNull(metrics);
-      assertRegistered(scope, metrics, metricManager);
+      assertRegistered(scope, metrics, coreMetricManager);
     } catch (final IllegalArgumentException e) {
       assertTrue("expected at least one null but got: scope="+scope+" category="+category+" metrics="+metrics,
           (scope == null || category == null || metrics == null));
-      assertRegistered(scope, new HashMap<>(), metricManager);
+      assertRegistered(scope, new HashMap<>(), coreMetricManager);
     }
   }
 
@@ -89,10 +91,10 @@ public class SolrCoreMetricManagerTest extends SolrTestCaseJ4 {
       if (metrics.isEmpty()) {
         continue;
       }
-      SolrMetricProducer producer = SolrMetricTestUtils.getProducerOf(category, scope, metrics);
-      metricManager.registerMetricProducer(scope, producer);
+      SolrMetricProducer producer = SolrMetricTestUtils.getProducerOf(metricManager, category, scope, metrics);
+      coreMetricManager.registerMetricProducer(scope, producer);
       registered.putAll(metrics);
-      assertRegistered(scope, registered, metricManager);
+      assertRegistered(scope, registered, coreMetricManager);
     }
   }
 
@@ -115,24 +117,24 @@ public class SolrCoreMetricManagerTest extends SolrTestCaseJ4 {
     PluginInfo pluginInfo = shouldDefinePlugin ? new PluginInfo(TestUtil.randomUnicodeString(random), attrs) : null;
 
     try {
-      SolrMetricManager.loadReporter(metricManager.getRegistryName(), metricManager.getCore().getResourceLoader(), pluginInfo);
+      metricManager.loadReporter(coreMetricManager.getRegistryName(), coreMetricManager.getCore().getResourceLoader(), pluginInfo);
       assertNotNull(pluginInfo);
-      Map<String, SolrMetricReporter> reporters = SolrMetricManager.getReporters(metricManager.getRegistryName());
+      Map<String, SolrMetricReporter> reporters = metricManager.getReporters(coreMetricManager.getRegistryName());
       assertTrue("reporters.size should be > 0, but was + " + reporters.size(), reporters.size() > 0);
       assertNotNull("reporter " + reporterName + " not present among " + reporters, reporters.get(reporterName));
       assertTrue("wrong reporter class: " + reporters.get(reporterName), reporters.get(reporterName) instanceof MockMetricReporter);
     } catch (IllegalArgumentException e) {
       assertTrue(pluginInfo == null || attrs.get("configurable") == null);
-      assertNull(SolrMetricManager.getReporters(metricManager.getRegistryName()).get(reporterName));
+      assertNull(metricManager.getReporters(coreMetricManager.getRegistryName()).get(reporterName));
     }
   }
 
-  private static void assertRegistered(String scope, Map<String, Counter> newMetrics, SolrCoreMetricManager metricManager) {
+  private void assertRegistered(String scope, Map<String, Counter> newMetrics, SolrCoreMetricManager coreMetricManager) {
     if (scope == null) {
       return;
     }
     String filter = "." + scope + ".";
-    MetricRegistry registry = SolrMetricManager.registry(metricManager.getRegistryName());
+    MetricRegistry registry = metricManager.registry(coreMetricManager.getRegistryName());
     assertEquals(newMetrics.size(), registry.getMetrics().
         keySet().stream().filter(s -> s.contains(filter)).count());
 
@@ -159,12 +161,12 @@ public class SolrCoreMetricManagerTest extends SolrTestCaseJ4 {
     String cloudRegistryName = "solr.core." + cloudCoreName;
     String nestedRegistryName = "solr.core.my_collection_.shard1_0.replica0";
     // pass through
-    assertEquals(cloudRegistryName, metricManager.createRegistryName(null, cloudCoreName));
-    assertEquals(simpleRegistryName, metricManager.createRegistryName(null, simpleCoreName));
+    assertEquals(cloudRegistryName, coreMetricManager.createRegistryName(null, cloudCoreName));
+    assertEquals(simpleRegistryName, coreMetricManager.createRegistryName(null, simpleCoreName));
     // unknown naming scheme -> pass through
-    assertEquals(simpleRegistryName, metricManager.createRegistryName(collectionName, simpleCoreName));
+    assertEquals(simpleRegistryName, coreMetricManager.createRegistryName(collectionName, simpleCoreName));
     // cloud collection
-    assertEquals(nestedRegistryName, metricManager.createRegistryName(collectionName, cloudCoreName));
+    assertEquals(nestedRegistryName, coreMetricManager.createRegistryName(collectionName, cloudCoreName));
 
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/adfb4a50/solr/core/src/test/org/apache/solr/metrics/SolrMetricManagerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/SolrMetricManagerTest.java b/solr/core/src/test/org/apache/solr/metrics/SolrMetricManagerTest.java
index 2886016..ecddfba 100644
--- a/solr/core/src/test/org/apache/solr/metrics/SolrMetricManagerTest.java
+++ b/solr/core/src/test/org/apache/solr/metrics/SolrMetricManagerTest.java
@@ -54,23 +54,25 @@ public class SolrMetricManagerTest extends SolrTestCaseJ4 {
   public void testMoveMetrics() throws Exception {
     Random r = random();
 
+    SolrMetricManager metricManager = new SolrMetricManager();
+
     Map<String, Counter> metrics1 = SolrMetricTestUtils.getRandomMetrics(r, true);
     Map<String, Counter> metrics2 = SolrMetricTestUtils.getRandomMetrics(r, true);
     String fromName = TestUtil.randomSimpleString(r, 1, 10);
     String toName = TestUtil.randomSimpleString(r, 1, 10);
     // register test metrics
     for (Map.Entry<String, Counter> entry : metrics1.entrySet()) {
-      SolrMetricManager.register(fromName, entry.getValue(), false, entry.getKey(), "metrics1");
+      metricManager.register(fromName, entry.getValue(), false, entry.getKey(), "metrics1");
     }
     for (Map.Entry<String, Counter> entry : metrics2.entrySet()) {
-      SolrMetricManager.register(fromName, entry.getValue(), false, entry.getKey(), "metrics2");
+      metricManager.register(fromName, entry.getValue(), false, entry.getKey(), "metrics2");
     }
-    assertEquals(metrics1.size() + metrics2.size(), SolrMetricManager.registry(fromName).getMetrics().size());
+    assertEquals(metrics1.size() + metrics2.size(), metricManager.registry(fromName).getMetrics().size());
 
     // move metrics1
-    SolrMetricManager.moveMetrics(fromName, toName, new SolrMetricManager.PrefixFilter("metrics1"));
+    metricManager.moveMetrics(fromName, toName, new SolrMetricManager.PrefixFilter("metrics1"));
     // check the remaining metrics
-    Map<String, Metric> fromMetrics = SolrMetricManager.registry(fromName).getMetrics();
+    Map<String, Metric> fromMetrics = metricManager.registry(fromName).getMetrics();
     assertEquals(metrics2.size(), fromMetrics.size());
     for (Map.Entry<String, Counter> entry : metrics2.entrySet()) {
       Object value = fromMetrics.get(SolrMetricManager.mkName(entry.getKey(), "metrics2"));
@@ -78,7 +80,7 @@ public class SolrMetricManagerTest extends SolrTestCaseJ4 {
       assertEquals(entry.getValue(), value);
     }
     // check the moved metrics
-    Map<String, Metric> toMetrics = SolrMetricManager.registry(toName).getMetrics();
+    Map<String, Metric> toMetrics = metricManager.registry(toName).getMetrics();
     assertEquals(metrics1.size(), toMetrics.size());
     for (Map.Entry<String, Counter> entry : metrics1.entrySet()) {
       Object value = toMetrics.get(SolrMetricManager.mkName(entry.getKey(), "metrics1"));
@@ -87,10 +89,10 @@ public class SolrMetricManagerTest extends SolrTestCaseJ4 {
     }
 
     // move all remaining metrics
-    SolrMetricManager.moveMetrics(fromName, toName, null);
-    fromMetrics = SolrMetricManager.registry(fromName).getMetrics();
+    metricManager.moveMetrics(fromName, toName, null);
+    fromMetrics = metricManager.registry(fromName).getMetrics();
     assertEquals(0, fromMetrics.size());
-    toMetrics = SolrMetricManager.registry(toName).getMetrics();
+    toMetrics = metricManager.registry(toName).getMetrics();
     assertEquals(metrics1.size() + metrics2.size(), toMetrics.size());
   }
 
@@ -98,6 +100,8 @@ public class SolrMetricManagerTest extends SolrTestCaseJ4 {
   public void testRegisterAll() throws Exception {
     Random r = random();
 
+    SolrMetricManager metricManager = new SolrMetricManager();
+
     Map<String, Counter> metrics = SolrMetricTestUtils.getRandomMetrics(r, true);
     MetricRegistry mr = new MetricRegistry();
     for (Map.Entry<String, Counter> entry : metrics.entrySet()) {
@@ -105,13 +109,13 @@ public class SolrMetricManagerTest extends SolrTestCaseJ4 {
     }
 
     String registryName = TestUtil.randomSimpleString(r, 1, 10);
-    assertEquals(0, SolrMetricManager.registry(registryName).getMetrics().size());
-    SolrMetricManager.registerAll(registryName, mr, false);
+    assertEquals(0, metricManager.registry(registryName).getMetrics().size());
+    metricManager.registerAll(registryName, mr, false);
     // this should simply skip existing names
-    SolrMetricManager.registerAll(registryName, mr, true);
+    metricManager.registerAll(registryName, mr, true);
     // this should produce error
     try {
-      SolrMetricManager.registerAll(registryName, mr, false);
+      metricManager.registerAll(registryName, mr, false);
       fail("registerAll with duplicate metric names should fail");
     } catch (IllegalArgumentException e) {
       // expected
@@ -122,34 +126,36 @@ public class SolrMetricManagerTest extends SolrTestCaseJ4 {
   public void testClearMetrics() throws Exception {
     Random r = random();
 
+    SolrMetricManager metricManager = new SolrMetricManager();
+
     Map<String, Counter> metrics = SolrMetricTestUtils.getRandomMetrics(r, true);
     String registryName = TestUtil.randomSimpleString(r, 1, 10);
 
     for (Map.Entry<String, Counter> entry : metrics.entrySet()) {
-      SolrMetricManager.register(registryName, entry.getValue(), false, entry.getKey(), "foo", "bar");
+      metricManager.register(registryName, entry.getValue(), false, entry.getKey(), "foo", "bar");
     }
     for (Map.Entry<String, Counter> entry : metrics.entrySet()) {
-      SolrMetricManager.register(registryName, entry.getValue(), false, entry.getKey(), "foo", "baz");
+      metricManager.register(registryName, entry.getValue(), false, entry.getKey(), "foo", "baz");
     }
     for (Map.Entry<String, Counter> entry : metrics.entrySet()) {
-      SolrMetricManager.register(registryName, entry.getValue(), false, entry.getKey(), "foo");
+      metricManager.register(registryName, entry.getValue(), false, entry.getKey(), "foo");
     }
 
-    assertEquals(metrics.size() * 3, SolrMetricManager.registry(registryName).getMetrics().size());
+    assertEquals(metrics.size() * 3, metricManager.registry(registryName).getMetrics().size());
 
     // clear "foo.bar"
-    Set<String> removed = SolrMetricManager.clearMetrics(registryName, "foo", "bar");
+    Set<String> removed = metricManager.clearMetrics(registryName, "foo", "bar");
     assertEquals(metrics.size(), removed.size());
     for (String s : removed) {
       assertTrue(s.startsWith("foo.bar."));
     }
-    removed = SolrMetricManager.clearMetrics(registryName, "foo", "baz");
+    removed = metricManager.clearMetrics(registryName, "foo", "baz");
     assertEquals(metrics.size(), removed.size());
     for (String s : removed) {
       assertTrue(s.startsWith("foo.baz."));
     }
     // perhaps surprisingly, this works too - see PrefixFilter docs
-    removed = SolrMetricManager.clearMetrics(registryName, "fo");
+    removed = metricManager.clearMetrics(registryName, "fo");
     assertEquals(metrics.size(), removed.size());
     for (String s : removed) {
       assertTrue(s.startsWith("foo."));
@@ -160,13 +166,15 @@ public class SolrMetricManagerTest extends SolrTestCaseJ4 {
   public void testSimpleMetrics() throws Exception {
     Random r = random();
 
+    SolrMetricManager metricManager = new SolrMetricManager();
+
     String registryName = TestUtil.randomSimpleString(r, 1, 10);
 
-    SolrMetricManager.counter(registryName, "simple_counter", "foo", "bar");
-    SolrMetricManager.timer(registryName, "simple_timer", "foo", "bar");
-    SolrMetricManager.meter(registryName, "simple_meter", "foo", "bar");
-    SolrMetricManager.histogram(registryName, "simple_histogram", "foo", "bar");
-    Map<String, Metric> metrics = SolrMetricManager.registry(registryName).getMetrics();
+    metricManager.counter(registryName, "simple_counter", "foo", "bar");
+    metricManager.timer(registryName, "simple_timer", "foo", "bar");
+    metricManager.meter(registryName, "simple_meter", "foo", "bar");
+    metricManager.histogram(registryName, "simple_histogram", "foo", "bar");
+    Map<String, Metric> metrics = metricManager.registry(registryName).getMetrics();
     assertEquals(4, metrics.size());
     for (Map.Entry<String, Metric> entry : metrics.entrySet()) {
       assertTrue(entry.getKey().startsWith("foo.bar.simple_"));
@@ -194,6 +202,7 @@ public class SolrMetricManagerTest extends SolrTestCaseJ4 {
     Random r = random();
 
     SolrResourceLoader loader = new SolrResourceLoader();
+    SolrMetricManager metricManager = new SolrMetricManager();
 
     PluginInfo[] plugins = new PluginInfo[] {
         createPluginInfo("universal_foo", null, null),
@@ -204,8 +213,8 @@ public class SolrMetricManagerTest extends SolrTestCaseJ4 {
         createPluginInfo("core_foo", "core", null)
     };
 
-    SolrMetricManager.loadReporters(plugins, loader, SolrInfoMBean.Group.node);
-    Map<String, SolrMetricReporter> reporters = SolrMetricManager.getReporters(
+    metricManager.loadReporters(plugins, loader, SolrInfoMBean.Group.node);
+    Map<String, SolrMetricReporter> reporters = metricManager.getReporters(
         SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node));
     assertEquals(4, reporters.size());
     assertTrue(reporters.containsKey("universal_foo"));
@@ -213,8 +222,8 @@ public class SolrMetricManagerTest extends SolrTestCaseJ4 {
     assertTrue(reporters.containsKey("node_foo"));
     assertTrue(reporters.containsKey("multiregistry_foo"));
 
-    SolrMetricManager.loadReporters(plugins, loader, SolrInfoMBean.Group.core, "collection1");
-    reporters = SolrMetricManager.getReporters(
+    metricManager.loadReporters(plugins, loader, SolrInfoMBean.Group.core, "collection1");
+    reporters = metricManager.getReporters(
         SolrMetricManager.getRegistryName(SolrInfoMBean.Group.core, "collection1"));
     assertEquals(5, reporters.size());
     assertTrue(reporters.containsKey("universal_foo"));
@@ -223,25 +232,25 @@ public class SolrMetricManagerTest extends SolrTestCaseJ4 {
     assertTrue(reporters.containsKey("core_foo"));
     assertTrue(reporters.containsKey("multiregistry_foo"));
 
-    SolrMetricManager.loadReporters(plugins, loader, SolrInfoMBean.Group.jvm);
-    reporters = SolrMetricManager.getReporters(
+    metricManager.loadReporters(plugins, loader, SolrInfoMBean.Group.jvm);
+    reporters = metricManager.getReporters(
         SolrMetricManager.getRegistryName(SolrInfoMBean.Group.jvm));
     assertEquals(2, reporters.size());
     assertTrue(reporters.containsKey("universal_foo"));
     assertTrue(reporters.containsKey("multigroup_foo"));
 
-    SolrMetricManager.removeRegistry("solr.jvm");
-    reporters = SolrMetricManager.getReporters(
+    metricManager.removeRegistry("solr.jvm");
+    reporters = metricManager.getReporters(
         SolrMetricManager.getRegistryName(SolrInfoMBean.Group.jvm));
     assertEquals(0, reporters.size());
 
-    SolrMetricManager.removeRegistry("solr.node");
-    reporters = SolrMetricManager.getReporters(
+    metricManager.removeRegistry("solr.node");
+    reporters = metricManager.getReporters(
         SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node));
     assertEquals(0, reporters.size());
 
-    SolrMetricManager.removeRegistry("solr.core.collection1");
-    reporters = SolrMetricManager.getReporters(
+    metricManager.removeRegistry("solr.core.collection1");
+    reporters = metricManager.getReporters(
         SolrMetricManager.getRegistryName(SolrInfoMBean.Group.core, "collection1"));
     assertEquals(0, reporters.size());
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/adfb4a50/solr/core/src/test/org/apache/solr/metrics/SolrMetricReporterTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/SolrMetricReporterTest.java b/solr/core/src/test/org/apache/solr/metrics/SolrMetricReporterTest.java
index 0cbf009..b275919 100644
--- a/solr/core/src/test/org/apache/solr/metrics/SolrMetricReporterTest.java
+++ b/solr/core/src/test/org/apache/solr/metrics/SolrMetricReporterTest.java
@@ -34,8 +34,10 @@ public class SolrMetricReporterTest extends LuceneTestCase {
   public void testInit() throws Exception {
     Random random = random();
 
+    SolrMetricManager metricManager = new SolrMetricManager();
+
     final String registryName = TestUtil.randomSimpleString(random);
-    final MockMetricReporter reporter = new MockMetricReporter(registryName);
+    final MockMetricReporter reporter = new MockMetricReporter(metricManager, registryName);
 
     Map<String, Object> attrs = new HashMap<>();
     attrs.put(FieldType.CLASS_NAME, MockMetricReporter.class.getName());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/adfb4a50/solr/core/src/test/org/apache/solr/metrics/SolrMetricTestUtils.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/SolrMetricTestUtils.java b/solr/core/src/test/org/apache/solr/metrics/SolrMetricTestUtils.java
index f8ca6d7..44ae9db 100644
--- a/solr/core/src/test/org/apache/solr/metrics/SolrMetricTestUtils.java
+++ b/solr/core/src/test/org/apache/solr/metrics/SolrMetricTestUtils.java
@@ -79,15 +79,15 @@ public final class SolrMetricTestUtils {
     return metrics;
   }
 
-  public static SolrMetricProducer getProducerOf(SolrInfoMBean.Category category, String scope, Map<String, Counter> metrics) {
+  public static SolrMetricProducer getProducerOf(SolrMetricManager metricManager, SolrInfoMBean.Category category, String scope, Map<String, Counter> metrics) {
     return new SolrMetricProducer() {
       @Override
-      public Collection<String> initializeMetrics(String registry, String scope) {
+      public Collection<String> initializeMetrics(SolrMetricManager manager, String registry, String scope) {
         if (metrics == null || metrics.isEmpty()) {
           return Collections.emptyList();
         }
         for (Map.Entry<String, Counter> entry : metrics.entrySet()) {
-          SolrMetricManager.counter(registry, entry.getKey(), category.toString(), scope);
+          manager.counter(registry, entry.getKey(), category.toString(), scope);
         }
         return metrics.keySet();
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/adfb4a50/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java b/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java
index 67a073b..c6449ac 100644
--- a/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java
@@ -54,6 +54,7 @@ public class SolrMetricsIntegrationTest extends SolrTestCaseJ4 {
   private static final SolrInfoMBean.Category HANDLER_CATEGORY = SolrInfoMBean.Category.QUERYHANDLER;
 
   private CoreContainer cc;
+  private SolrMetricManager metricManager;
 
   @Before
   public void beforeTest() throws Exception {
@@ -66,8 +67,9 @@ public class SolrMetricsIntegrationTest extends SolrTestCaseJ4 {
     cc = createCoreContainer(cfg,
         new TestHarness.TestCoresLocator(DEFAULT_TEST_CORENAME, initCoreDataDir.getAbsolutePath(), "solrconfig.xml", "schema.xml"));
     h.coreName = DEFAULT_TEST_CORENAME;
+    metricManager = cc.getMetricManager();
     // initially there are more reporters, because two of them are added via a matching collection name
-    Map<String, SolrMetricReporter> reporters = SolrMetricManager.getReporters("solr.core." + DEFAULT_TEST_CORENAME);
+    Map<String, SolrMetricReporter> reporters = metricManager.getReporters("solr.core." + DEFAULT_TEST_CORENAME);
     assertEquals(INITIAL_REPORTERS.length, reporters.size());
     assertTrue(reporters.keySet().containsAll(Arrays.asList(INITIAL_REPORTERS)));
     // test rename operation
@@ -77,7 +79,7 @@ public class SolrMetricsIntegrationTest extends SolrTestCaseJ4 {
     PluginInfo[] plugins = cfg.getMetricReporterPlugins();
     assertNotNull(plugins);
     assertEquals(10, plugins.length);
-    reporters = SolrMetricManager.getReporters("solr.node");
+    reporters = metricManager.getReporters("solr.node");
     assertEquals(4, reporters.size());
     assertTrue("Reporter '" + REPORTER_NAMES[0] + "' missing in solr.node", reporters.containsKey(REPORTER_NAMES[0]));
     assertTrue("Reporter '" + UNIVERSAL + "' missing in solr.node", reporters.containsKey(UNIVERSAL));
@@ -93,8 +95,8 @@ public class SolrMetricsIntegrationTest extends SolrTestCaseJ4 {
 
   @After
   public void afterTest() throws Exception {
-    SolrCoreMetricManager metricManager = h.getCore().getMetricManager();
-    Map<String, SolrMetricReporter> reporters = SolrMetricManager.getReporters(metricManager.getRegistryName());
+    SolrCoreMetricManager coreMetricManager = h.getCore().getCoreMetricManager();
+    Map<String, SolrMetricReporter> reporters = metricManager.getReporters(coreMetricManager.getRegistryName());
 
     deleteCore();
 
@@ -110,8 +112,8 @@ public class SolrMetricsIntegrationTest extends SolrTestCaseJ4 {
     Random random = random();
 
     String metricName = SolrMetricManager.mkName(METRIC_NAME, HANDLER_CATEGORY.toString(), HANDLER_NAME);
-    SolrCoreMetricManager metricManager = h.getCore().getMetricManager();
-    Timer timer = (Timer) SolrMetricManager.timer(metricManager.getRegistryName(), metricName);
+    SolrCoreMetricManager coreMetricManager = h.getCore().getCoreMetricManager();
+    Timer timer = (Timer) metricManager.timer(coreMetricManager.getRegistryName(), metricName);
 
     long initialCount = timer.getCount();
 
@@ -122,7 +124,7 @@ public class SolrMetricsIntegrationTest extends SolrTestCaseJ4 {
 
     long finalCount = timer.getCount();
     assertEquals("metric counter incorrect", iterations, finalCount - initialCount);
-    Map<String, SolrMetricReporter> reporters = SolrMetricManager.getReporters(metricManager.getRegistryName());
+    Map<String, SolrMetricReporter> reporters = metricManager.getReporters(coreMetricManager.getRegistryName());
     assertEquals(RENAMED_REPORTERS.length, reporters.size());
 
     // SPECIFIC and MULTIREGISTRY were skipped because they were

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/adfb4a50/solr/core/src/test/org/apache/solr/metrics/reporters/MockMetricReporter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/reporters/MockMetricReporter.java b/solr/core/src/test/org/apache/solr/metrics/reporters/MockMetricReporter.java
index 4d5ef69..2ecc33b 100644
--- a/solr/core/src/test/org/apache/solr/metrics/reporters/MockMetricReporter.java
+++ b/solr/core/src/test/org/apache/solr/metrics/reporters/MockMetricReporter.java
@@ -34,8 +34,8 @@ public class MockMetricReporter extends SolrMetricReporter {
   public boolean didClose = false;
   public boolean didValidate = false;
 
-  public MockMetricReporter(String registryName) {
-    super(registryName);
+  public MockMetricReporter(SolrMetricManager metricManager, String registryName) {
+    super(metricManager, registryName);
   }
 
   @Override
@@ -62,7 +62,7 @@ public class MockMetricReporter extends SolrMetricReporter {
   }
 
   public Metric reportMetric(String metricName) throws NoSuchElementException {
-    MetricRegistry registry = SolrMetricManager.registry(registryName);
+    MetricRegistry registry = metricManager.registry(registryName);
     Metric metric = registry.getMetrics().get(metricName);
     if (metric == null) {
       throw new NoSuchElementException("Metric was not found for metric name = " + metricName);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/adfb4a50/solr/core/src/test/org/apache/solr/metrics/reporters/SolrJmxReporterTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/reporters/SolrJmxReporterTest.java b/solr/core/src/test/org/apache/solr/metrics/reporters/SolrJmxReporterTest.java
index 607296e..ea452b2 100644
--- a/solr/core/src/test/org/apache/solr/metrics/reporters/SolrJmxReporterTest.java
+++ b/solr/core/src/test/org/apache/solr/metrics/reporters/SolrJmxReporterTest.java
@@ -48,7 +48,8 @@ public class SolrJmxReporterTest extends SolrTestCaseJ4 {
 
   private String domain;
 
-  private SolrCoreMetricManager metricManager;
+  private SolrCoreMetricManager coreMetricManager;
+  private SolrMetricManager metricManager;
   private SolrJmxReporter reporter;
   private MBeanServer mBeanServer;
   private String reporterName;
@@ -60,11 +61,12 @@ public class SolrJmxReporterTest extends SolrTestCaseJ4 {
     final SolrCore core = h.getCore();
     domain = core.getName();
 
-    metricManager = new SolrCoreMetricManager(core);
+    coreMetricManager = core.getCoreMetricManager();
+    metricManager = core.getCoreDescriptor().getCoreContainer().getMetricManager();
     PluginInfo pluginInfo = createReporterPluginInfo();
-    SolrMetricManager.loadReporter(metricManager.getRegistryName(), metricManager.getCore().getResourceLoader(), pluginInfo);
+    metricManager.loadReporter(coreMetricManager.getRegistryName(), coreMetricManager.getCore().getResourceLoader(), pluginInfo);
 
-    Map<String, SolrMetricReporter> reporters = SolrMetricManager.getReporters(metricManager.getRegistryName());
+    Map<String, SolrMetricReporter> reporters = metricManager.getReporters(coreMetricManager.getRegistryName());
     assertTrue("reporters.size should be > 0, but was + " + reporters.size(), reporters.size() > 0);
     reporterName = pluginInfo.name;
     assertNotNull("reporter " + reporterName + " not present among " + reporters, reporters.get(reporterName));
@@ -95,12 +97,12 @@ public class SolrJmxReporterTest extends SolrTestCaseJ4 {
 
   @After
   public void afterTest() throws Exception {
-    SolrMetricManager.closeReporters(metricManager.getRegistryName());
+    metricManager.closeReporters(coreMetricManager.getRegistryName());
     Set<ObjectInstance> objects =
         mBeanServer.queryMBeans(ObjectName.getInstance(domain + ":*"), null);
     assertTrue(objects.isEmpty());
 
-    metricManager.close();
+    coreMetricManager.close();
     deleteCore();
   }
 
@@ -115,8 +117,8 @@ public class SolrJmxReporterTest extends SolrTestCaseJ4 {
     int iterations = TestUtil.nextInt(random, 0, MAX_ITERATIONS);
     for (int i = 0; i < iterations; ++i) {
       Map<String, Counter> metrics = SolrMetricTestUtils.getRandomMetricsWithReplacements(random, registered);
-      SolrMetricProducer producer = SolrMetricTestUtils.getProducerOf(category, scope, metrics);
-      metricManager.registerMetricProducer(scope, producer);
+      SolrMetricProducer producer = SolrMetricTestUtils.getProducerOf(metricManager, category, scope, metrics);
+      coreMetricManager.registerMetricProducer(scope, producer);
       registered.putAll(metrics);
       //waitForListener();
       Set<ObjectInstance> objects = mBeanServer.queryMBeans(null, null);
@@ -133,8 +135,8 @@ public class SolrJmxReporterTest extends SolrTestCaseJ4 {
     String scope = SolrMetricTestUtils.getRandomScope(random, true);
     SolrInfoMBean.Category category = SolrMetricTestUtils.getRandomCategory(random, true);
     Map<String, Counter> metrics = SolrMetricTestUtils.getRandomMetrics(random, true);
-    SolrMetricProducer producer = SolrMetricTestUtils.getProducerOf(category, scope, metrics);
-    metricManager.registerMetricProducer(scope, producer);
+    SolrMetricProducer producer = SolrMetricTestUtils.getProducerOf(metricManager, category, scope, metrics);
+    coreMetricManager.registerMetricProducer(scope, producer);
     Set<ObjectInstance> objects = mBeanServer.queryMBeans(null, null);
     assertEquals(metrics.size(), objects.stream().
         filter(o -> scope.equals(o.getObjectName().getKeyProperty("scope")) &&
@@ -142,8 +144,8 @@ public class SolrJmxReporterTest extends SolrTestCaseJ4 {
 
     h.getCoreContainer().reload(h.getCore().getName());
     PluginInfo pluginInfo = createReporterPluginInfo();
-    SolrMetricManager.loadReporter(metricManager.getRegistryName(), metricManager.getCore().getResourceLoader(), pluginInfo);
-    metricManager.registerMetricProducer(scope, producer);
+    metricManager.loadReporter(coreMetricManager.getRegistryName(), coreMetricManager.getCore().getResourceLoader(), pluginInfo);
+    coreMetricManager.registerMetricProducer(scope, producer);
 
     objects = mBeanServer.queryMBeans(null, null);
     assertEquals(metrics.size(), objects.stream().

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/adfb4a50/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java b/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
index 66992a7..4e6f6d6 100644
--- a/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
+++ b/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
@@ -192,21 +192,17 @@ public class TestHarness extends BaseTestHarness {
         = new UpdateShardHandlerConfig(UpdateShardHandlerConfig.DEFAULT_MAXUPDATECONNECTIONS,
                                        UpdateShardHandlerConfig.DEFAULT_MAXUPDATECONNECTIONSPERHOST,
                                        30000, 30000);
-    // metric reporters
+    // universal default metric reporter
     Map<String,String> attributes = new HashMap<>();
-    attributes.put("group", SolrInfoMBean.Group.core.toString());
     attributes.put("name", "default");
     attributes.put("class", SolrJmxReporter.class.getName());
-    PluginInfo corePlugin = new PluginInfo("reporter", attributes, null, null);
-    attributes = new HashMap<>(attributes);
-    attributes.put("group", SolrInfoMBean.Group.node.toString());
-    PluginInfo nodePlugin = new PluginInfo("reporter", attributes, null, null);
+    PluginInfo defaultPlugin = new PluginInfo("reporter", attributes, null, null);
 
     return new NodeConfig.NodeConfigBuilder("testNode", loader)
         .setUseSchemaCache(Boolean.getBoolean("shareSchema"))
         .setCloudConfig(cloudConfig)
         .setUpdateShardHandlerConfig(updateShardHandlerConfig)
-        .setMetricReporterPlugins(new PluginInfo[] {corePlugin, nodePlugin})
+        .setMetricReporterPlugins(new PluginInfo[] {defaultPlugin})
         .build();
   }
 


[30/44] lucene-solr:jira/solr-9854: SOLR-9513: Generic Hadoop authentication plugins, GenericHadoopAuthPlugin and ConfigurableInternodeAuthHadoopPlugin

Posted by ab...@apache.org.
SOLR-9513: Generic Hadoop authentication plugins, GenericHadoopAuthPlugin and ConfigurableInternodeAuthHadoopPlugin


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

Branch: refs/heads/jira/solr-9854
Commit: a1a8b2864e621c18aa86b21d4a244233e991a47d
Parents: 321c6f0
Author: Ishan Chattopadhyaya <is...@apache.org>
Authored: Mon Dec 19 20:45:04 2016 +0530
Committer: Ishan Chattopadhyaya <is...@apache.org>
Committed: Mon Dec 19 20:45:04 2016 +0530

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../security/AttributeOnlyServletContext.java   | 291 ++++++++++++++
 .../ConfigurableInternodeAuthHadoopPlugin.java  |  68 ++++
 .../security/DelegationTokenKerberosFilter.java |   6 +-
 .../solr/security/GenericHadoopAuthPlugin.java  | 266 ++++++++++++
 .../apache/solr/security/HadoopAuthFilter.java  | 198 +++++++++
 .../apache/solr/security/HadoopAuthPlugin.java  | 241 +++++++++++
 .../apache/solr/security/KerberosFilter.java    |   6 +-
 .../apache/solr/security/KerberosPlugin.java    | 314 +--------------
 ...tContinuesRecorderAuthenticationHandler.java |  71 ++++
 .../solr/security/hadoop_kerberos_config.json   |  16 +
 .../hadoop_simple_auth_with_delegation.json     |  29 ++
 .../TestSolrCloudWithSecureImpersonation.java   |   8 +-
 .../solr/security/hadoop/ImpersonationUtil.java |  73 ++++
 .../hadoop/ImpersonatorCollectionsHandler.java  |  60 +++
 .../hadoop/TestDelegationWithHadoopAuth.java    | 400 +++++++++++++++++++
 .../hadoop/TestImpersonationWithHadoopAuth.java | 215 ++++++++++
 .../TestSolrCloudWithHadoopAuthPlugin.java      | 136 +++++++
 .../solrj/impl/HttpClientBuilderFactory.java    |  41 ++
 .../solrj/impl/Krb5HttpClientBuilder.java       |  10 +-
 .../apache/solr/cloud/MiniSolrCloudCluster.java |  29 +-
 .../apache/solr/cloud/SolrCloudTestCase.java    |  30 +-
 22 files changed, 2185 insertions(+), 326 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index c920575..f783934 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -173,6 +173,9 @@ New Features
 * SOLR-9844: FieldCache information fetched via the mbeans handler or seen via the UI now displays the total size used.
   The individual cache entries in the response are now formatted better as well. (Varun Thacker)
 
+* SOLR-9513: Generic authentication plugins (GenericHadoopAuthPlugin and ConfigurableInternodeAuthHadoopPlugin) that delegate
+  all functionality to Hadoop authentication framework. (Hrishikesh Gadre via Ishan Chattopadhyaya)
+
 Optimizations
 ----------------------
 * SOLR-9704: Facet Module / JSON Facet API: Optimize blockChildren facets that have

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/core/src/java/org/apache/solr/security/AttributeOnlyServletContext.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/AttributeOnlyServletContext.java b/solr/core/src/java/org/apache/solr/security/AttributeOnlyServletContext.java
new file mode 100644
index 0000000..4abcd40
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/security/AttributeOnlyServletContext.java
@@ -0,0 +1,291 @@
+/*
+ * 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.security;
+
+import java.io.InputStream;
+import java.net.MalformedURLException;
+import java.net.URL;
+
+import java.util.Collections;
+import java.util.Enumeration;
+import java.util.EventListener;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+import javax.servlet.Filter;
+import javax.servlet.FilterRegistration;
+import javax.servlet.RequestDispatcher;
+import javax.servlet.Servlet;
+import javax.servlet.ServletContext;
+import javax.servlet.ServletException;
+import javax.servlet.ServletRegistration;
+import javax.servlet.SessionCookieConfig;
+import javax.servlet.SessionTrackingMode;
+import javax.servlet.FilterRegistration.Dynamic;
+import javax.servlet.descriptor.JspConfigDescriptor;
+
+/**
+ * A concrete implementation of {@linkplain ServletContext} which support only attributes.
+ */
+class AttributeOnlyServletContext implements ServletContext {
+  private Map<String, Object> attributes = new HashMap<String, Object>();
+
+  @Override
+  public void setSessionTrackingModes(Set<SessionTrackingMode> sessionTrackingModes) {}
+
+  @Override
+  public boolean setInitParameter(String name, String value) {
+    return false;
+  }
+
+  @Override
+  public void setAttribute(String name, Object object) {
+    attributes.put(name, object);
+  }
+
+  @Override
+  public void removeAttribute(String name) {
+    attributes.remove(name);
+  }
+
+  @Override
+  public void log(String message, Throwable throwable) {}
+
+  @Override
+  public void log(Exception exception, String msg) {}
+
+  @Override
+  public void log(String msg) {}
+
+  @Override
+  public String getVirtualServerName() {
+    return null;
+  }
+
+  @Override
+  public SessionCookieConfig getSessionCookieConfig() {
+    return null;
+  }
+
+  @Override
+  public Enumeration<Servlet> getServlets() {
+    return null;
+  }
+
+  @Override
+  public Map<String,? extends ServletRegistration> getServletRegistrations() {
+    return null;
+  }
+
+  @Override
+  public ServletRegistration getServletRegistration(String servletName) {
+    return null;
+  }
+
+  @Override
+  public Enumeration<String> getServletNames() {
+    return null;
+  }
+
+  @Override
+  public String getServletContextName() {
+    return null;
+  }
+
+  @Override
+  public Servlet getServlet(String name) throws ServletException {
+    return null;
+  }
+
+  @Override
+  public String getServerInfo() {
+    return null;
+  }
+
+  @Override
+  public Set<String> getResourcePaths(String path) {
+    return null;
+  }
+
+  @Override
+  public InputStream getResourceAsStream(String path) {
+    return null;
+  }
+
+  @Override
+  public URL getResource(String path) throws MalformedURLException {
+    return null;
+  }
+
+  @Override
+  public RequestDispatcher getRequestDispatcher(String path) {
+    return null;
+  }
+
+  @Override
+  public String getRealPath(String path) {
+    return null;
+  }
+
+  @Override
+  public RequestDispatcher getNamedDispatcher(String name) {
+    return null;
+  }
+
+  @Override
+  public int getMinorVersion() {
+    return 0;
+  }
+
+  @Override
+  public String getMimeType(String file) {
+    return null;
+  }
+
+  @Override
+  public int getMajorVersion() {
+    return 0;
+  }
+
+  @Override
+  public JspConfigDescriptor getJspConfigDescriptor() {
+    return null;
+  }
+
+  @Override
+  public Enumeration<String> getInitParameterNames() {
+    return null;
+  }
+
+  @Override
+  public String getInitParameter(String name) {
+    return null;
+  }
+
+  @Override
+  public Map<String,? extends FilterRegistration> getFilterRegistrations() {
+    return null;
+  }
+
+  @Override
+  public FilterRegistration getFilterRegistration(String filterName) {
+    return null;
+  }
+
+  @Override
+  public Set<SessionTrackingMode> getEffectiveSessionTrackingModes() {
+    return null;
+  }
+
+  @Override
+  public int getEffectiveMinorVersion() {
+    return 0;
+  }
+
+  @Override
+  public int getEffectiveMajorVersion() {
+    return 0;
+  }
+
+  @Override
+  public Set<SessionTrackingMode> getDefaultSessionTrackingModes() {
+    return null;
+  }
+
+  @Override
+  public String getContextPath() {
+    return null;
+  }
+
+  @Override
+  public ServletContext getContext(String uripath) {
+    return null;
+  }
+
+  @Override
+  public ClassLoader getClassLoader() {
+    return null;
+  }
+
+  @Override
+  public Enumeration<String> getAttributeNames() {
+    return Collections.enumeration(attributes.keySet());
+  }
+
+  @Override
+  public Object getAttribute(String name) {
+    return attributes.get(name);
+  }
+
+  @Override
+  public void declareRoles(String... roleNames) {}
+
+  @Override
+  public <T extends Servlet> T createServlet(Class<T> clazz) throws ServletException {
+    return null;
+  }
+
+  @Override
+  public <T extends EventListener> T createListener(Class<T> clazz) throws ServletException {
+    return null;
+  }
+
+  @Override
+  public <T extends Filter> T createFilter(Class<T> clazz) throws ServletException {
+    return null;
+  }
+
+  @Override
+  public javax.servlet.ServletRegistration.Dynamic addServlet(String servletName, Class<? extends Servlet> servletClass) {
+    return null;
+  }
+
+  @Override
+  public javax.servlet.ServletRegistration.Dynamic addServlet(String servletName, Servlet servlet) {
+    return null;
+  }
+
+  @Override
+  public javax.servlet.ServletRegistration.Dynamic addServlet(String servletName, String className) {
+    return null;
+  }
+
+  @Override
+  public void addListener(Class<? extends EventListener> listenerClass) {}
+
+  @Override
+  public <T extends EventListener> void addListener(T t) {}
+
+  @Override
+  public void addListener(String className) {}
+
+  @Override
+  public Dynamic addFilter(String filterName, Class<? extends Filter> filterClass) {
+    return null;
+  }
+
+  @Override
+  public Dynamic addFilter(String filterName, Filter filter) {
+    return null;
+  }
+
+  @Override
+  public Dynamic addFilter(String filterName, String className) {
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/core/src/java/org/apache/solr/security/ConfigurableInternodeAuthHadoopPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/ConfigurableInternodeAuthHadoopPlugin.java b/solr/core/src/java/org/apache/solr/security/ConfigurableInternodeAuthHadoopPlugin.java
new file mode 100644
index 0000000..f3bb70f
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/security/ConfigurableInternodeAuthHadoopPlugin.java
@@ -0,0 +1,68 @@
+/*
+ * 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.security;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+
+import org.apache.solr.client.solrj.impl.HttpClientBuilderFactory;
+import org.apache.solr.client.solrj.impl.SolrHttpClientBuilder;
+import org.apache.solr.core.CoreContainer;
+
+/**
+ * This class extends {@linkplain HadoopAuthPlugin} by enabling configuration of
+ * authentication mechanism for Solr internal communication.
+ **/
+public class ConfigurableInternodeAuthHadoopPlugin extends HadoopAuthPlugin implements HttpClientBuilderPlugin {
+
+  /**
+   * A property specifying the {@linkplain HttpClientBuilderFactory} used for the Solr internal
+   * communication.
+   */
+  private static final String HTTPCLIENT_BUILDER_FACTORY = "clientBuilderFactory";
+
+  private HttpClientBuilderFactory factory = null;
+
+  public ConfigurableInternodeAuthHadoopPlugin(CoreContainer coreContainer) {
+    super(coreContainer);
+  }
+
+  @Override
+  public void init(Map<String,Object> pluginConfig) {
+    super.init(pluginConfig);
+
+    String httpClientBuilderFactory = (String)Objects.requireNonNull(pluginConfig.get(HTTPCLIENT_BUILDER_FACTORY),
+        "Please specify clientBuilderFactory to be used for Solr internal communication.");
+    factory = this.coreContainer.getResourceLoader().newInstance(httpClientBuilderFactory, HttpClientBuilderFactory.class);
+  }
+
+  @Override
+  public SolrHttpClientBuilder getHttpClientBuilder(SolrHttpClientBuilder builder) {
+    return factory.getHttpClientBuilder(Optional.ofNullable(builder));
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close();
+
+    if (factory != null) {
+      factory.close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/core/src/java/org/apache/solr/security/DelegationTokenKerberosFilter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/DelegationTokenKerberosFilter.java b/solr/core/src/java/org/apache/solr/security/DelegationTokenKerberosFilter.java
index 421de52..007e0bd 100644
--- a/solr/core/src/java/org/apache/solr/security/DelegationTokenKerberosFilter.java
+++ b/solr/core/src/java/org/apache/solr/security/DelegationTokenKerberosFilter.java
@@ -141,9 +141,9 @@ public class DelegationTokenKerberosFilter extends DelegationTokenAuthentication
     // set the internal authentication handler in order to record whether the request should continue
     super.initializeAuthHandler(authHandlerClassName, filterConfig);
     AuthenticationHandler authHandler = getAuthenticationHandler();
-    super.initializeAuthHandler(KerberosPlugin.RequestContinuesRecorderAuthenticationHandler.class.getName(), filterConfig);
-    KerberosPlugin.RequestContinuesRecorderAuthenticationHandler newAuthHandler =
-        (KerberosPlugin.RequestContinuesRecorderAuthenticationHandler)getAuthenticationHandler();
+    super.initializeAuthHandler(RequestContinuesRecorderAuthenticationHandler.class.getName(), filterConfig);
+    RequestContinuesRecorderAuthenticationHandler newAuthHandler =
+        (RequestContinuesRecorderAuthenticationHandler)getAuthenticationHandler();
     newAuthHandler.setAuthHandler(authHandler);
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/core/src/java/org/apache/solr/security/GenericHadoopAuthPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/GenericHadoopAuthPlugin.java b/solr/core/src/java/org/apache/solr/security/GenericHadoopAuthPlugin.java
new file mode 100644
index 0000000..e5fe349
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/security/GenericHadoopAuthPlugin.java
@@ -0,0 +1,266 @@
+/*
+ * 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.security;
+
+import static org.apache.solr.security.RequestContinuesRecorderAuthenticationHandler.REQUEST_CONTINUES_ATTR;
+import static org.apache.solr.security.HadoopAuthFilter.DELEGATION_TOKEN_ZK_CLIENT;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.lang.invoke.MethodHandles;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+
+import javax.servlet.FilterChain;
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletContext;
+import javax.servlet.ServletException;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletResponse;
+import javax.servlet.http.HttpServletResponseWrapper;
+
+import org.apache.commons.collections.iterators.IteratorEnumeration;
+import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
+import org.apache.solr.client.solrj.impl.HttpClientBuilderFactory;
+import org.apache.solr.client.solrj.impl.Krb5HttpClientBuilder;
+import org.apache.solr.client.solrj.impl.SolrHttpClientBuilder;
+import org.apache.solr.cloud.ZkController;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.util.SuppressForbidden;
+import org.apache.solr.core.CoreContainer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class implements a generic plugin which can use authentication schemes exposed by the
+ * Hadoop framework. This plugin supports following features
+ * - integration with authentication mehcanisms (e.g. kerberos)
+ * - Delegation token support
+ * - Proxy users (or secure impersonation) support
+ *
+ * This plugin enables defining configuration parameters required by the undelying Hadoop authentication
+ * mechanism. These configuration parameters can either be specified as a Java system property or the default
+ * value can be specified as part of the plugin configuration.
+ *
+ * The proxy users are configured by specifying relevant Hadoop configuration parameters. Please note that
+ * the delegation token support must be enabled for using the proxy users support.
+ *
+ * For Solr internal communication, this plugin enables configuring {@linkplain HttpClientBuilderFactory}
+ * implementation (e.g. based on kerberos).
+ */
+public class GenericHadoopAuthPlugin extends AuthenticationPlugin implements HttpClientBuilderPlugin {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  /**
+   * A property specifying the type of authentication scheme to be configured.
+   */
+  private static final String HADOOP_AUTH_TYPE = "type";
+
+  /**
+   * A property specifies the value of the prefix to be used to define Java system property
+   * for configuring the authentication mechanism. The name of the Java system property is
+   * defined by appending the configuration parmeter namne to this prefix value e.g. if prefix
+   * is 'solr' then the Java system property 'solr.kerberos.principal' defines the value of
+   * configuration parameter 'kerberos.principal'.
+   */
+  private static final String SYSPROP_PREFIX_PROPERTY = "sysPropPrefix";
+
+  /**
+   * A property specifying the configuration parameters required by the authentication scheme
+   * defined by {@linkplain #HADOOP_AUTH_TYPE} property.
+   */
+  private static final String AUTH_CONFIG_NAMES_PROPERTY = "authConfigs";
+
+  /**
+   * A property specifying the {@linkplain HttpClientBuilderFactory} used for the Solr internal
+   * communication.
+   */
+  private static final String HTTPCLIENT_BUILDER_FACTORY = "clientBuilderFactory";
+
+  /**
+   * A property specifying the default values for the configuration parameters specified by the
+   * {@linkplain #AUTH_CONFIG_NAMES_PROPERTY} property. The default values are specified as a
+   * collection of key-value pairs (i.e. property-name : default_value).
+   */
+  private static final String DEFAULT_AUTH_CONFIGS_PROPERTY = "defaultConfigs";
+
+  /**
+   * A property which enable (or disable) the delegation tokens functionality.
+   */
+  private static final String DELEGATION_TOKEN_ENABLED_PROPERTY = "enableDelegationToken";
+
+  /**
+   * A property which enables initialization of kerberos before connecting to Zookeeper.
+   */
+  private static final String INIT_KERBEROS_ZK = "initKerberosZk";
+
+  /**
+   * A property which configures proxy users for the underlying Hadoop authentication mechanism.
+   * This configuration is expressed as a collection of key-value pairs  (i.e. property-name : value).
+   */
+  public static final String PROXY_USER_CONFIGS = "proxyUserConfigs";
+
+  private AuthenticationFilter authFilter;
+  private HttpClientBuilderFactory factory = null;
+  private final CoreContainer coreContainer;
+
+  public GenericHadoopAuthPlugin(CoreContainer coreContainer) {
+    this.coreContainer = coreContainer;
+  }
+
+  @SuppressWarnings("rawtypes")
+  @Override
+  public void init(Map<String,Object> pluginConfig) {
+    try {
+      String delegationTokenEnabled = (String)pluginConfig.getOrDefault(DELEGATION_TOKEN_ENABLED_PROPERTY, "false");
+      authFilter = (Boolean.parseBoolean(delegationTokenEnabled)) ? new HadoopAuthFilter() : new AuthenticationFilter();
+
+      // Initialize kerberos before initializing curator instance.
+      boolean initKerberosZk = Boolean.parseBoolean((String)pluginConfig.getOrDefault(INIT_KERBEROS_ZK, "false"));
+      if (initKerberosZk) {
+        (new Krb5HttpClientBuilder()).getBuilder();
+      }
+
+      FilterConfig conf = getInitFilterConfig(pluginConfig);
+      authFilter.init(conf);
+
+      String httpClientBuilderFactory = (String)pluginConfig.get(HTTPCLIENT_BUILDER_FACTORY);
+      if (httpClientBuilderFactory != null) {
+        Class c = Class.forName(httpClientBuilderFactory);
+        factory = (HttpClientBuilderFactory)c.newInstance();
+      }
+
+    } catch (ServletException | ClassNotFoundException | InstantiationException | IllegalAccessException e) {
+      throw new SolrException(ErrorCode.SERVER_ERROR, "Error initializing kerberos authentication plugin: "+e);
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  protected FilterConfig getInitFilterConfig(Map<String, Object> pluginConfig) {
+    Map<String, String> params = new HashMap<>();
+
+    String type = (String) Objects.requireNonNull(pluginConfig.get(HADOOP_AUTH_TYPE));
+    params.put(HADOOP_AUTH_TYPE, type);
+
+    String sysPropPrefix = (String) pluginConfig.getOrDefault(SYSPROP_PREFIX_PROPERTY, "solr.");
+    Collection<String> authConfigNames = (Collection<String>) pluginConfig.
+        getOrDefault(AUTH_CONFIG_NAMES_PROPERTY, Collections.emptyList());
+    Map<String,String> authConfigDefaults = (Map<String,String>) pluginConfig
+        .getOrDefault(DEFAULT_AUTH_CONFIGS_PROPERTY, Collections.emptyMap());
+    Map<String,String> proxyUserConfigs = (Map<String,String>) pluginConfig
+        .getOrDefault(PROXY_USER_CONFIGS, Collections.emptyMap());
+
+    for ( String configName : authConfigNames) {
+      String systemProperty = sysPropPrefix + configName;
+      String defaultConfigVal = authConfigDefaults.get(configName);
+      String configVal = System.getProperty(systemProperty, defaultConfigVal);
+      if (configVal != null) {
+        params.put(configName, configVal);
+      }
+    }
+
+    // Configure proxy user settings.
+    params.putAll(proxyUserConfigs);
+
+    final ServletContext servletContext = new AttributeOnlyServletContext();
+    log.info("Params: "+params);
+
+    ZkController controller = coreContainer.getZkController();
+    if (controller != null) {
+      servletContext.setAttribute(DELEGATION_TOKEN_ZK_CLIENT, controller.getZkClient());
+    }
+
+    FilterConfig conf = new FilterConfig() {
+      @Override
+      public ServletContext getServletContext() {
+        return servletContext;
+      }
+
+      @Override
+      public Enumeration<String> getInitParameterNames() {
+        return new IteratorEnumeration(params.keySet().iterator());
+      }
+
+      @Override
+      public String getInitParameter(String param) {
+        return params.get(param);
+      }
+
+      @Override
+      public String getFilterName() {
+        return "HadoopAuthFilter";
+      }
+    };
+
+    return conf;
+  }
+
+  @Override
+  public boolean doAuthenticate(ServletRequest request, ServletResponse response, FilterChain filterChain)
+      throws Exception {
+    final HttpServletResponse frsp = (HttpServletResponse)response;
+
+    // Workaround until HADOOP-13346 is fixed.
+    HttpServletResponse rspCloseShield = new HttpServletResponseWrapper(frsp) {
+      @SuppressForbidden(reason = "Hadoop DelegationTokenAuthenticationFilter uses response writer, this" +
+          "is providing a CloseShield on top of that")
+      @Override
+      public PrintWriter getWriter() throws IOException {
+        final PrintWriter pw = new PrintWriterWrapper(frsp.getWriter()) {
+          @Override
+          public void close() {};
+        };
+        return pw;
+      }
+    };
+    authFilter.doFilter(request, rspCloseShield, filterChain);
+
+    if (authFilter instanceof HadoopAuthFilter) { // delegation token mgmt.
+      String requestContinuesAttr = (String)request.getAttribute(REQUEST_CONTINUES_ATTR);
+      if (requestContinuesAttr == null) {
+        log.warn("Could not find " + REQUEST_CONTINUES_ATTR);
+        return false;
+      } else {
+        return Boolean.parseBoolean(requestContinuesAttr);
+      }
+    }
+
+    return true;
+  }
+
+  @Override
+  public SolrHttpClientBuilder getHttpClientBuilder(SolrHttpClientBuilder builder) {
+    return (factory != null) ? factory.getHttpClientBuilder(Optional.ofNullable(builder)) : builder;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (authFilter != null) {
+      authFilter.destroy();
+    }
+    if (factory != null) {
+      factory.close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/core/src/java/org/apache/solr/security/HadoopAuthFilter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/HadoopAuthFilter.java b/solr/core/src/java/org/apache/solr/security/HadoopAuthFilter.java
new file mode 100644
index 0000000..fb35e72
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/security/HadoopAuthFilter.java
@@ -0,0 +1,198 @@
+/*
+ * 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.security;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+
+import javax.servlet.FilterChain;
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletException;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletRequestWrapper;
+
+import org.apache.curator.RetryPolicy;
+import org.apache.curator.framework.AuthInfo;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.api.ACLProvider;
+import org.apache.curator.retry.ExponentialBackoffRetry;
+
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.server.AuthenticationHandler;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticationFilter;
+import org.apache.hadoop.security.token.delegation.web.HttpUserGroupInformation;
+import org.apache.solr.common.cloud.SecurityAwareZkACLProvider;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.cloud.ZkACLProvider;
+import org.apache.solr.common.cloud.ZkCredentialsProvider;
+import org.apache.zookeeper.data.ACL;
+
+/**
+ * This is an authentication filter based on Hadoop's {@link DelegationTokenAuthenticationFilter}.
+ */
+public class HadoopAuthFilter extends DelegationTokenAuthenticationFilter {
+  /**
+   * This property defines the configuration parameter storing the Solr zookeeper client ref
+   * in the servlet filter config.
+   */
+  static final String DELEGATION_TOKEN_ZK_CLIENT = "solr.kerberos.delegation.token.zk.client";
+
+  private CuratorFramework curatorFramework;
+
+  @Override
+  public void init(FilterConfig conf) throws ServletException {
+    if (conf != null && "zookeeper".equals(conf.getInitParameter("signer.secret.provider"))) {
+      SolrZkClient zkClient =
+          (SolrZkClient)conf.getServletContext().getAttribute(DELEGATION_TOKEN_ZK_CLIENT);
+      conf.getServletContext().setAttribute("signer.secret.provider.zookeeper.curator.client",
+          getCuratorClient(zkClient));
+    }
+    super.init(conf);
+  }
+
+  @Override
+  public void doFilter(ServletRequest request, ServletResponse response,
+      FilterChain filterChain) throws IOException, ServletException {
+    // HttpClient 4.4.x throws NPE if query string is null and parsed through URLEncodedUtils.
+    // See HTTPCLIENT-1746 and HADOOP-12767
+    HttpServletRequest httpRequest = (HttpServletRequest)request;
+    String queryString = httpRequest.getQueryString();
+    final String nonNullQueryString = queryString == null ? "" : queryString;
+    HttpServletRequest requestNonNullQueryString = new HttpServletRequestWrapper(httpRequest){
+      @Override
+      public String getQueryString() {
+        return nonNullQueryString;
+      }
+    };
+
+    // include Impersonator User Name in case someone (e.g. logger) wants it
+    FilterChain filterChainWrapper = new FilterChain() {
+      @Override
+      public void doFilter(ServletRequest servletRequest, ServletResponse servletResponse)
+          throws IOException, ServletException {
+        HttpServletRequest httpRequest = (HttpServletRequest) servletRequest;
+
+        UserGroupInformation ugi = HttpUserGroupInformation.get();
+        if (ugi != null && ugi.getAuthenticationMethod() == UserGroupInformation.AuthenticationMethod.PROXY) {
+          UserGroupInformation realUserUgi = ugi.getRealUser();
+          if (realUserUgi != null) {
+            httpRequest.setAttribute(KerberosPlugin.IMPERSONATOR_USER_NAME, realUserUgi.getShortUserName());
+          }
+        }
+        filterChain.doFilter(servletRequest, servletResponse);
+      }
+    };
+
+    super.doFilter(requestNonNullQueryString, response, filterChainWrapper);
+  }
+
+  @Override
+  public void destroy() {
+    super.destroy();
+    if (curatorFramework != null) {
+      curatorFramework.close();
+    }
+    curatorFramework = null;
+  }
+
+  @Override
+  protected void initializeAuthHandler(String authHandlerClassName,
+                                       FilterConfig filterConfig) throws ServletException {
+    // set the internal authentication handler in order to record whether the request should continue
+    super.initializeAuthHandler(authHandlerClassName, filterConfig);
+    AuthenticationHandler authHandler = getAuthenticationHandler();
+    super.initializeAuthHandler(RequestContinuesRecorderAuthenticationHandler.class.getName(), filterConfig);
+    RequestContinuesRecorderAuthenticationHandler newAuthHandler =
+        (RequestContinuesRecorderAuthenticationHandler)getAuthenticationHandler();
+    newAuthHandler.setAuthHandler(authHandler);
+  }
+
+  protected CuratorFramework getCuratorClient(SolrZkClient zkClient) {
+    // should we try to build a RetryPolicy off of the ZkController?
+    RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3);
+    if (zkClient == null) {
+      throw new IllegalArgumentException("zkClient required");
+    }
+    String zkHost = zkClient.getZkServerAddress();
+    String zkChroot = zkHost.contains("/")? zkHost.substring(zkHost.indexOf("/")): "";
+    String zkNamespace = zkChroot + SecurityAwareZkACLProvider.SECURITY_ZNODE_PATH;
+    zkNamespace = zkNamespace.startsWith("/") ? zkNamespace.substring(1) : zkNamespace;
+    String zkConnectionString = zkHost.contains("/")? zkHost.substring(0, zkHost.indexOf("/")): zkHost;
+    SolrZkToCuratorCredentialsACLs curatorToSolrZk = new SolrZkToCuratorCredentialsACLs(zkClient);
+    final int connectionTimeoutMs = 30000; // this value is currently hard coded, see SOLR-7561.
+
+    curatorFramework = CuratorFrameworkFactory.builder()
+        .namespace(zkNamespace)
+        .connectString(zkConnectionString)
+        .retryPolicy(retryPolicy)
+        .aclProvider(curatorToSolrZk.getACLProvider())
+        .authorization(curatorToSolrZk.getAuthInfos())
+        .sessionTimeoutMs(zkClient.getZkClientTimeout())
+        .connectionTimeoutMs(connectionTimeoutMs)
+        .build();
+    curatorFramework.start();
+    return curatorFramework;
+  }
+
+  /**
+   * Convert Solr Zk Credentials/ACLs to Curator versions
+   */
+  protected static class SolrZkToCuratorCredentialsACLs {
+    private final ACLProvider aclProvider;
+    private final List<AuthInfo> authInfos;
+
+    public SolrZkToCuratorCredentialsACLs(SolrZkClient zkClient) {
+      this.aclProvider = createACLProvider(zkClient);
+      this.authInfos = createAuthInfo(zkClient);
+    }
+
+    public ACLProvider getACLProvider() { return aclProvider; }
+    public List<AuthInfo> getAuthInfos() { return authInfos; }
+
+    private ACLProvider createACLProvider(SolrZkClient zkClient) {
+      final ZkACLProvider zkACLProvider = zkClient.getZkACLProvider();
+      return new ACLProvider() {
+        @Override
+        public List<ACL> getDefaultAcl() {
+          return zkACLProvider.getACLsToAdd(null);
+        }
+
+        @Override
+        public List<ACL> getAclForPath(String path) {
+           List<ACL> acls = zkACLProvider.getACLsToAdd(path);
+           return acls;
+        }
+      };
+    }
+
+    private List<AuthInfo> createAuthInfo(SolrZkClient zkClient) {
+      List<AuthInfo> ret = new LinkedList<AuthInfo>();
+
+      // In theory the credentials to add could change here if zookeeper hasn't been initialized
+      ZkCredentialsProvider credentialsProvider =
+        zkClient.getZkClientConnectionStrategy().getZkCredentialsToAddAutomatically();
+      for (ZkCredentialsProvider.ZkCredentials zkCredentials : credentialsProvider.getCredentials()) {
+        ret.add(new AuthInfo(zkCredentials.getScheme(), zkCredentials.getAuth()));
+      }
+      return ret;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/core/src/java/org/apache/solr/security/HadoopAuthPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/HadoopAuthPlugin.java b/solr/core/src/java/org/apache/solr/security/HadoopAuthPlugin.java
new file mode 100644
index 0000000..db0f639
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/security/HadoopAuthPlugin.java
@@ -0,0 +1,241 @@
+/*
+ * 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.security;
+
+import static org.apache.solr.security.RequestContinuesRecorderAuthenticationHandler.REQUEST_CONTINUES_ATTR;
+import static org.apache.solr.security.HadoopAuthFilter.DELEGATION_TOKEN_ZK_CLIENT;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.lang.invoke.MethodHandles;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import javax.servlet.FilterChain;
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletContext;
+import javax.servlet.ServletException;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletResponse;
+import javax.servlet.http.HttpServletResponseWrapper;
+
+import org.apache.commons.collections.iterators.IteratorEnumeration;
+import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
+import org.apache.solr.client.solrj.impl.Krb5HttpClientBuilder;
+import org.apache.solr.cloud.ZkController;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.util.SuppressForbidden;
+import org.apache.solr.core.CoreContainer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class implements a generic plugin which can use authentication schemes exposed by the
+ * Hadoop framework. This plugin supports following features
+ * - integration with authentication mehcanisms (e.g. kerberos)
+ * - Delegation token support
+ * - Proxy users (or secure impersonation) support
+ *
+ * This plugin enables defining configuration parameters required by the undelying Hadoop authentication
+ * mechanism. These configuration parameters can either be specified as a Java system property or the default
+ * value can be specified as part of the plugin configuration.
+ *
+ * The proxy users are configured by specifying relevant Hadoop configuration parameters. Please note that
+ * the delegation token support must be enabled for using the proxy users support.
+ *
+ * Note - this class does not support configuring authentication mechanism for Solr internal communication.
+ * For this purpose {@linkplain ConfigurableInternodeAuthHadoopPlugin} should be used. If this plugin is used in the
+ * SolrCloud mode, it will use PKI based authentication mechanism for Solr internal communication.
+ **/
+public class HadoopAuthPlugin extends AuthenticationPlugin {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  /**
+   * A property specifying the type of authentication scheme to be configured.
+   */
+  private static final String HADOOP_AUTH_TYPE = "type";
+
+  /**
+   * A property specifies the value of the prefix to be used to define Java system property
+   * for configuring the authentication mechanism. The name of the Java system property is
+   * defined by appending the configuration parmeter namne to this prefix value e.g. if prefix
+   * is 'solr' then the Java system property 'solr.kerberos.principal' defines the value of
+   * configuration parameter 'kerberos.principal'.
+   */
+  private static final String SYSPROP_PREFIX_PROPERTY = "sysPropPrefix";
+
+  /**
+   * A property specifying the configuration parameters required by the authentication scheme
+   * defined by {@linkplain #HADOOP_AUTH_TYPE} property.
+   */
+  private static final String AUTH_CONFIG_NAMES_PROPERTY = "authConfigs";
+
+  /**
+   * A property specifying the default values for the configuration parameters specified by the
+   * {@linkplain #AUTH_CONFIG_NAMES_PROPERTY} property. The default values are specified as a
+   * collection of key-value pairs (i.e. property-name : default_value).
+   */
+  private static final String DEFAULT_AUTH_CONFIGS_PROPERTY = "defaultConfigs";
+
+  /**
+   * A property which enable (or disable) the delegation tokens functionality.
+   */
+  private static final String DELEGATION_TOKEN_ENABLED_PROPERTY = "enableDelegationToken";
+
+  /**
+   * A property which enables initialization of kerberos before connecting to Zookeeper.
+   */
+  private static final String INIT_KERBEROS_ZK = "initKerberosZk";
+
+  /**
+   * A property which configures proxy users for the underlying Hadoop authentication mechanism.
+   * This configuration is expressed as a collection of key-value pairs  (i.e. property-name : value).
+   */
+  public static final String PROXY_USER_CONFIGS = "proxyUserConfigs";
+
+  private AuthenticationFilter authFilter;
+  protected final CoreContainer coreContainer;
+
+  public HadoopAuthPlugin(CoreContainer coreContainer) {
+    this.coreContainer = coreContainer;
+  }
+
+  @Override
+  public void init(Map<String,Object> pluginConfig) {
+    try {
+      String delegationTokenEnabled = (String)pluginConfig.getOrDefault(DELEGATION_TOKEN_ENABLED_PROPERTY, "false");
+      authFilter = (Boolean.parseBoolean(delegationTokenEnabled)) ? new HadoopAuthFilter() : new AuthenticationFilter();
+
+      // Initialize kerberos before initializing curator instance.
+      boolean initKerberosZk = Boolean.parseBoolean((String)pluginConfig.getOrDefault(INIT_KERBEROS_ZK, "false"));
+      if (initKerberosZk) {
+        (new Krb5HttpClientBuilder()).getBuilder();
+      }
+
+      FilterConfig conf = getInitFilterConfig(pluginConfig);
+      authFilter.init(conf);
+
+    } catch (ServletException e) {
+      throw new SolrException(ErrorCode.SERVER_ERROR, "Error initializing GenericHadoopAuthPlugin: "+e);
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  protected FilterConfig getInitFilterConfig(Map<String, Object> pluginConfig) {
+    Map<String, String> params = new HashMap<>();
+
+    String type = (String) Objects.requireNonNull(pluginConfig.get(HADOOP_AUTH_TYPE));
+    params.put(HADOOP_AUTH_TYPE, type);
+
+    String sysPropPrefix = (String) pluginConfig.getOrDefault(SYSPROP_PREFIX_PROPERTY, "solr.");
+    Collection<String> authConfigNames = (Collection<String>) pluginConfig.
+        getOrDefault(AUTH_CONFIG_NAMES_PROPERTY, Collections.emptyList());
+    Map<String,String> authConfigDefaults = (Map<String,String>) pluginConfig
+        .getOrDefault(DEFAULT_AUTH_CONFIGS_PROPERTY, Collections.emptyMap());
+    Map<String,String> proxyUserConfigs = (Map<String,String>) pluginConfig
+        .getOrDefault(PROXY_USER_CONFIGS, Collections.emptyMap());
+
+    for ( String configName : authConfigNames) {
+      String systemProperty = sysPropPrefix + configName;
+      String defaultConfigVal = authConfigDefaults.get(configName);
+      String configVal = System.getProperty(systemProperty, defaultConfigVal);
+      if (configVal != null) {
+        params.put(configName, configVal);
+      }
+    }
+
+    // Configure proxy user settings.
+    params.putAll(proxyUserConfigs);
+
+    final ServletContext servletContext = new AttributeOnlyServletContext();
+    log.info("Params: "+params);
+
+    ZkController controller = coreContainer.getZkController();
+    if (controller != null) {
+      servletContext.setAttribute(DELEGATION_TOKEN_ZK_CLIENT, controller.getZkClient());
+    }
+
+    FilterConfig conf = new FilterConfig() {
+      @Override
+      public ServletContext getServletContext() {
+        return servletContext;
+      }
+
+      @Override
+      public Enumeration<String> getInitParameterNames() {
+        return new IteratorEnumeration(params.keySet().iterator());
+      }
+
+      @Override
+      public String getInitParameter(String param) {
+        return params.get(param);
+      }
+
+      @Override
+      public String getFilterName() {
+        return "HadoopAuthFilter";
+      }
+    };
+
+    return conf;
+  }
+
+  @Override
+  public boolean doAuthenticate(ServletRequest request, ServletResponse response, FilterChain filterChain)
+      throws Exception {
+    final HttpServletResponse frsp = (HttpServletResponse)response;
+
+    // Workaround until HADOOP-13346 is fixed.
+    HttpServletResponse rspCloseShield = new HttpServletResponseWrapper(frsp) {
+      @SuppressForbidden(reason = "Hadoop DelegationTokenAuthenticationFilter uses response writer, this" +
+          "is providing a CloseShield on top of that")
+      @Override
+      public PrintWriter getWriter() throws IOException {
+        final PrintWriter pw = new PrintWriterWrapper(frsp.getWriter()) {
+          @Override
+          public void close() {};
+        };
+        return pw;
+      }
+    };
+    authFilter.doFilter(request, rspCloseShield, filterChain);
+
+    if (authFilter instanceof HadoopAuthFilter) { // delegation token mgmt.
+      String requestContinuesAttr = (String)request.getAttribute(REQUEST_CONTINUES_ATTR);
+      if (requestContinuesAttr == null) {
+        log.warn("Could not find " + REQUEST_CONTINUES_ATTR);
+        return false;
+      } else {
+        return Boolean.parseBoolean(requestContinuesAttr);
+      }
+    }
+
+    return true;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (authFilter != null) {
+      authFilter.destroy();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/core/src/java/org/apache/solr/security/KerberosFilter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/KerberosFilter.java b/solr/core/src/java/org/apache/solr/security/KerberosFilter.java
index 9c53050..d725d09 100644
--- a/solr/core/src/java/org/apache/solr/security/KerberosFilter.java
+++ b/solr/core/src/java/org/apache/solr/security/KerberosFilter.java
@@ -42,9 +42,9 @@ public class KerberosFilter extends AuthenticationFilter {
     super.initializeAuthHandler(authHandlerClassName, filterConfig);
     AuthenticationHandler authHandler = getAuthenticationHandler();
     super.initializeAuthHandler(
-        KerberosPlugin.RequestContinuesRecorderAuthenticationHandler.class.getName(), filterConfig);
-    KerberosPlugin.RequestContinuesRecorderAuthenticationHandler newAuthHandler =
-        (KerberosPlugin.RequestContinuesRecorderAuthenticationHandler)getAuthenticationHandler();
+        RequestContinuesRecorderAuthenticationHandler.class.getName(), filterConfig);
+    RequestContinuesRecorderAuthenticationHandler newAuthHandler =
+        (RequestContinuesRecorderAuthenticationHandler)getAuthenticationHandler();
     newAuthHandler.setAuthHandler(authHandler);
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/core/src/java/org/apache/solr/security/KerberosPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/KerberosPlugin.java b/solr/core/src/java/org/apache/solr/security/KerberosPlugin.java
index 3655ac9..42d22ca 100644
--- a/solr/core/src/java/org/apache/solr/security/KerberosPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/KerberosPlugin.java
@@ -17,43 +17,24 @@
 package org.apache.solr.security;
 
 import java.io.IOException;
-import java.io.InputStream;
 import java.io.PrintWriter;
 import java.lang.invoke.MethodHandles;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.util.Collections;
 import java.util.Enumeration;
-import java.util.EventListener;
 import java.util.HashMap;
 import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
 
 import javax.servlet.Filter;
 import javax.servlet.FilterChain;
 import javax.servlet.FilterConfig;
-import javax.servlet.FilterRegistration;
-import javax.servlet.FilterRegistration.Dynamic;
-import javax.servlet.RequestDispatcher;
-import javax.servlet.Servlet;
 import javax.servlet.ServletContext;
 import javax.servlet.ServletException;
-import javax.servlet.ServletRegistration;
 import javax.servlet.ServletRequest;
 import javax.servlet.ServletResponse;
-import javax.servlet.SessionCookieConfig;
-import javax.servlet.SessionTrackingMode;
-import javax.servlet.descriptor.JspConfigDescriptor;
-import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 import javax.servlet.http.HttpServletResponseWrapper;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.collections.iterators.IteratorEnumeration;
-import org.apache.hadoop.security.authentication.client.AuthenticationException;
-import org.apache.hadoop.security.authentication.server.AuthenticationHandler;
-import org.apache.hadoop.security.authentication.server.AuthenticationToken;
 import org.apache.solr.client.solrj.impl.Krb5HttpClientBuilder;
 import org.apache.solr.client.solrj.impl.SolrHttpClientBuilder;
 import org.apache.solr.cloud.ZkController;
@@ -92,9 +73,6 @@ public class KerberosPlugin extends AuthenticationPlugin implements HttpClientBu
   public static final String IMPERSONATOR_DO_AS_HTTP_PARAM = "doAs";
   public static final String IMPERSONATOR_USER_NAME = "solr.impersonator.user.name";
 
-  // filled in by Plugin/Filter
-  static final String REQUEST_CONTINUES_ATTR =
-      "org.apache.solr.security.kerberosplugin.requestcontinues";
   static final String DELEGATION_TOKEN_ZK_CLIENT =
       "solr.kerberos.delegation.token.zk.client";
 
@@ -263,9 +241,9 @@ public class KerberosPlugin extends AuthenticationPlugin implements HttpClientBu
       }
     };
     kerberosFilter.doFilter(req, rspCloseShield, chain);
-    String requestContinuesAttr = (String)req.getAttribute(REQUEST_CONTINUES_ATTR);
+    String requestContinuesAttr = (String)req.getAttribute(RequestContinuesRecorderAuthenticationHandler.REQUEST_CONTINUES_ATTR);
     if (requestContinuesAttr == null) {
-      log.warn("Could not find " + REQUEST_CONTINUES_ATTR);
+      log.warn("Could not find " + RequestContinuesRecorderAuthenticationHandler.REQUEST_CONTINUES_ATTR);
       return false;
     } else {
       return Boolean.parseBoolean(requestContinuesAttr);
@@ -286,292 +264,4 @@ public class KerberosPlugin extends AuthenticationPlugin implements HttpClientBu
   protected Filter getKerberosFilter() { return kerberosFilter; }
 
   protected void setKerberosFilter(Filter kerberosFilter) { this.kerberosFilter = kerberosFilter; }
-
-  protected static class AttributeOnlyServletContext implements ServletContext {
-    private Map<String, Object> attributes = new HashMap<String, Object>();
-
-    @Override
-    public void setSessionTrackingModes(Set<SessionTrackingMode> sessionTrackingModes) {}
-    
-    @Override
-    public boolean setInitParameter(String name, String value) {
-      return false;
-    }
-
-    @Override
-    public void setAttribute(String name, Object object) {
-      attributes.put(name, object);
-    }
-
-    @Override
-    public void removeAttribute(String name) {
-      attributes.remove(name);
-    }
-    
-    @Override
-    public void log(String message, Throwable throwable) {}
-    
-    @Override
-    public void log(Exception exception, String msg) {}
-    
-    @Override
-    public void log(String msg) {}
-    
-    @Override
-    public String getVirtualServerName() {
-      return null;
-    }
-    
-    @Override
-    public SessionCookieConfig getSessionCookieConfig() {
-      return null;
-    }
-    
-    @Override
-    public Enumeration<Servlet> getServlets() {
-      return null;
-    }
-    
-    @Override
-    public Map<String,? extends ServletRegistration> getServletRegistrations() {
-      return null;
-    }
-    
-    @Override
-    public ServletRegistration getServletRegistration(String servletName) {
-      return null;
-    }
-    
-    @Override
-    public Enumeration<String> getServletNames() {
-      return null;
-    }
-    
-    @Override
-    public String getServletContextName() {
-      return null;
-    }
-    
-    @Override
-    public Servlet getServlet(String name) throws ServletException {
-      return null;
-    }
-    
-    @Override
-    public String getServerInfo() {
-      return null;
-    }
-    
-    @Override
-    public Set<String> getResourcePaths(String path) {
-      return null;
-    }
-    
-    @Override
-    public InputStream getResourceAsStream(String path) {
-      return null;
-    }
-    
-    @Override
-    public URL getResource(String path) throws MalformedURLException {
-      return null;
-    }
-    
-    @Override
-    public RequestDispatcher getRequestDispatcher(String path) {
-      return null;
-    }
-    
-    @Override
-    public String getRealPath(String path) {
-      return null;
-    }
-    
-    @Override
-    public RequestDispatcher getNamedDispatcher(String name) {
-      return null;
-    }
-    
-    @Override
-    public int getMinorVersion() {
-      return 0;
-    }
-    
-    @Override
-    public String getMimeType(String file) {
-      return null;
-    }
-    
-    @Override
-    public int getMajorVersion() {
-      return 0;
-    }
-    
-    @Override
-    public JspConfigDescriptor getJspConfigDescriptor() {
-      return null;
-    }
-    
-    @Override
-    public Enumeration<String> getInitParameterNames() {
-      return null;
-    }
-    
-    @Override
-    public String getInitParameter(String name) {
-      return null;
-    }
-    
-    @Override
-    public Map<String,? extends FilterRegistration> getFilterRegistrations() {
-      return null;
-    }
-    
-    @Override
-    public FilterRegistration getFilterRegistration(String filterName) {
-      return null;
-    }
-    
-    @Override
-    public Set<SessionTrackingMode> getEffectiveSessionTrackingModes() {
-      return null;
-    }
-    
-    @Override
-    public int getEffectiveMinorVersion() {
-      return 0;
-    }
-    
-    @Override
-    public int getEffectiveMajorVersion() {
-      return 0;
-    }
-    
-    @Override
-    public Set<SessionTrackingMode> getDefaultSessionTrackingModes() {
-      return null;
-    }
-    
-    @Override
-    public String getContextPath() {
-      return null;
-    }
-    
-    @Override
-    public ServletContext getContext(String uripath) {
-      return null;
-    }
-    
-    @Override
-    public ClassLoader getClassLoader() {
-      return null;
-    }
-
-    @Override
-    public Enumeration<String> getAttributeNames() {
-      return Collections.enumeration(attributes.keySet());
-    }
-
-    @Override
-    public Object getAttribute(String name) {
-      return attributes.get(name);
-    }
-    
-    @Override
-    public void declareRoles(String... roleNames) {}
-    
-    @Override
-    public <T extends Servlet> T createServlet(Class<T> clazz) throws ServletException {
-      return null;
-    }
-    
-    @Override
-    public <T extends EventListener> T createListener(Class<T> clazz) throws ServletException {
-      return null;
-    }
-    
-    @Override
-    public <T extends Filter> T createFilter(Class<T> clazz) throws ServletException {
-      return null;
-    }
-    
-    @Override
-    public javax.servlet.ServletRegistration.Dynamic addServlet(String servletName, Class<? extends Servlet> servletClass) {
-      return null;
-    }
-    
-    @Override
-    public javax.servlet.ServletRegistration.Dynamic addServlet(String servletName, Servlet servlet) {
-      return null;
-    }
-    
-    @Override
-    public javax.servlet.ServletRegistration.Dynamic addServlet(String servletName, String className) {
-      return null;
-    }
-    
-    @Override
-    public void addListener(Class<? extends EventListener> listenerClass) {}
-    
-    @Override
-    public <T extends EventListener> void addListener(T t) {}
-    
-    @Override
-    public void addListener(String className) {}
-    
-    @Override
-    public Dynamic addFilter(String filterName, Class<? extends Filter> filterClass) {
-      return null;
-    }
-    
-    @Override
-    public Dynamic addFilter(String filterName, Filter filter) {
-      return null;
-    }
-    
-    @Override
-    public Dynamic addFilter(String filterName, String className) {
-      return null;
-    }
-  };
-
-  /*
-   * {@link AuthenticationHandler} that delegates to another {@link AuthenticationHandler}
-   * and records the response of managementOperation (which indicates whether the request
-   * should continue or not).
-   */
-  public static class RequestContinuesRecorderAuthenticationHandler implements AuthenticationHandler {
-    private AuthenticationHandler authHandler;
-
-    public void setAuthHandler(AuthenticationHandler authHandler) {
-      this.authHandler = authHandler;
-    }
-
-    public String getType() {
-      return authHandler.getType();
-    }
-
-    public void init(Properties config) throws ServletException {
-      // authHandler has already been init'ed, nothing to do here
-    }
-
-    public void destroy() {
-      authHandler.destroy();
-    }
-
-    public boolean managementOperation(AuthenticationToken token,
-                                       HttpServletRequest request,
-                                       HttpServletResponse response)
-        throws IOException, AuthenticationException {
-      boolean result = authHandler.managementOperation(token, request, response);
-      request.setAttribute(KerberosPlugin.REQUEST_CONTINUES_ATTR, new Boolean(result).toString());
-      return result;
-    }
-
-
-    public AuthenticationToken authenticate(HttpServletRequest request, HttpServletResponse response)
-        throws IOException, AuthenticationException {
-      return authHandler.authenticate(request, response);
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/core/src/java/org/apache/solr/security/RequestContinuesRecorderAuthenticationHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/RequestContinuesRecorderAuthenticationHandler.java b/solr/core/src/java/org/apache/solr/security/RequestContinuesRecorderAuthenticationHandler.java
new file mode 100644
index 0000000..95591ca
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/security/RequestContinuesRecorderAuthenticationHandler.java
@@ -0,0 +1,71 @@
+/*
+ * 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.security;
+
+import java.io.IOException;
+import java.util.Properties;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.hadoop.security.authentication.server.AuthenticationHandler;
+import org.apache.hadoop.security.authentication.server.AuthenticationToken;
+
+/*
+ * {@link AuthenticationHandler} that delegates to another {@link AuthenticationHandler}
+ * and records the response of managementOperation (which indicates whether the request
+ * should continue or not).
+ */
+public class RequestContinuesRecorderAuthenticationHandler implements AuthenticationHandler {
+  // filled in by Plugin/Filter
+  static final String REQUEST_CONTINUES_ATTR =
+      "org.apache.solr.security.authentication.requestcontinues";
+
+  private AuthenticationHandler authHandler;
+
+  public void setAuthHandler(AuthenticationHandler authHandler) {
+    this.authHandler = authHandler;
+  }
+
+  public String getType() {
+    return authHandler.getType();
+  }
+
+  public void init(Properties config) throws ServletException {
+    // authHandler has already been init'ed, nothing to do here
+  }
+
+  public void destroy() {
+    authHandler.destroy();
+  }
+
+  public boolean managementOperation(AuthenticationToken token,
+                                     HttpServletRequest request,
+                                     HttpServletResponse response)
+      throws IOException, AuthenticationException {
+    boolean result = authHandler.managementOperation(token, request, response);
+    request.setAttribute(RequestContinuesRecorderAuthenticationHandler.REQUEST_CONTINUES_ATTR, new Boolean(result).toString());
+    return result;
+  }
+
+  public AuthenticationToken authenticate(HttpServletRequest request, HttpServletResponse response)
+      throws IOException, AuthenticationException {
+    return authHandler.authenticate(request, response);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/core/src/test-files/solr/security/hadoop_kerberos_config.json
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/security/hadoop_kerberos_config.json b/solr/core/src/test-files/solr/security/hadoop_kerberos_config.json
new file mode 100644
index 0000000..679474e
--- /dev/null
+++ b/solr/core/src/test-files/solr/security/hadoop_kerberos_config.json
@@ -0,0 +1,16 @@
+{
+    "authentication": {
+        "class": "org.apache.solr.security.ConfigurableInternodeAuthHadoopPlugin",
+        "sysPropPrefix": "solr.",
+        "type": "kerberos",
+        "clientBuilderFactory": "org.apache.solr.client.solrj.impl.Krb5HttpClientBuilder",
+        "initKerberosZk": "true",
+        "authConfigs": [
+            "kerberos.principal",
+            "kerberos.keytab",
+            "kerberos.name.rules"
+        ],
+        "defaultConfigs": {
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/core/src/test-files/solr/security/hadoop_simple_auth_with_delegation.json
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/security/hadoop_simple_auth_with_delegation.json b/solr/core/src/test-files/solr/security/hadoop_simple_auth_with_delegation.json
new file mode 100644
index 0000000..2248329
--- /dev/null
+++ b/solr/core/src/test-files/solr/security/hadoop_simple_auth_with_delegation.json
@@ -0,0 +1,29 @@
+{
+    "authentication": {
+        "class": "org.apache.solr.security.HadoopAuthPlugin",
+        "sysPropPrefix": "solr.",
+        "type": "simple",
+        "enableDelegationToken":"true",
+        "authConfigs": [
+            "delegation-token.token-kind",
+            "delegation-token.update-interval.sec",
+            "delegation-token.max-lifetime.sec",
+            "delegation-token.renewal-interval.sec",
+            "delegation-token.removal-scan-interval.sec",
+            "cookie.domain",
+            "signer.secret.provider",
+            "zk-dt-secret-manager.enable",
+            "zk-dt-secret-manager.znodeWorkingPath",
+            "signer.secret.provider.zookeeper.path"
+        ],
+        "defaultConfigs": {
+            "delegation-token.token-kind": "solr-dt",
+            "signer.secret.provider": "zookeeper",
+            "zk-dt-secret-manager.enable": "true",
+            "token.validity": "36000",
+            "zk-dt-secret-manager.znodeWorkingPath": "solr/security/zkdtsm",
+            "signer.secret.provider.zookeeper.path": "/token",
+            "cookie.domain": "127.0.0.1"
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/core/src/test/org/apache/solr/cloud/TestSolrCloudWithSecureImpersonation.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestSolrCloudWithSecureImpersonation.java b/solr/core/src/test/org/apache/solr/cloud/TestSolrCloudWithSecureImpersonation.java
index 53b7f5b..d2f7873 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestSolrCloudWithSecureImpersonation.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestSolrCloudWithSecureImpersonation.java
@@ -108,11 +108,9 @@ public class TestSolrCloudWithSecureImpersonation extends SolrTestCaseJ4 {
     System.setProperty("solr.test.sys.prop2", "proptwo");
 
     SolrRequestParsers.DEFAULT.setAddRequestHeadersToContext(true);
-    String solrXml = MiniSolrCloudCluster.DEFAULT_CLOUD_SOLR_XML.replace("</solr>",
-        " <str name=\"collectionsHandler\">" + ImpersonatorCollectionsHandler.class.getName() + "</str>\n" +
-            "</solr>");
+    System.setProperty("collectionsHandler", ImpersonatorCollectionsHandler.class.getName());
 
-    miniCluster = new MiniSolrCloudCluster(NUM_SERVERS, createTempDir(), solrXml, buildJettyConfig("/solr"));
+    miniCluster = new MiniSolrCloudCluster(NUM_SERVERS, createTempDir(), buildJettyConfig("/solr"));
     JettySolrRunner runner = miniCluster.getJettySolrRunners().get(0);
     solrClient = new HttpSolrClient.Builder(runner.getBaseUrl().toString()).build();
   }
@@ -168,6 +166,8 @@ public class TestSolrCloudWithSecureImpersonation extends SolrTestCaseJ4 {
     }
     System.clearProperty("solr.test.sys.prop1");
     System.clearProperty("solr.test.sys.prop2");
+    System.clearProperty("collectionsHandler");
+
     SolrRequestParsers.DEFAULT.setAddRequestHeadersToContext(false);
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/core/src/test/org/apache/solr/security/hadoop/ImpersonationUtil.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/hadoop/ImpersonationUtil.java b/solr/core/src/test/org/apache/solr/security/hadoop/ImpersonationUtil.java
new file mode 100644
index 0000000..00c2b71
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/security/hadoop/ImpersonationUtil.java
@@ -0,0 +1,73 @@
+/*
+ * 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.security.hadoop;
+
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.authentication.client.PseudoAuthenticator;
+import org.apache.lucene.util.Constants;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.security.HadoopAuthPlugin;
+import org.apache.solr.security.KerberosPlugin;
+
+/**
+ * This class implements utility functions required to test the secure impersonation feature for {@linkplain HadoopAuthPlugin}
+ */
+public class ImpersonationUtil {
+
+  static String getUsersFirstGroup() throws Exception {
+    String group = "*"; // accept any group if a group can't be found
+    if (!Constants.WINDOWS) { // does not work on Windows!
+      org.apache.hadoop.security.Groups hGroups =
+          new org.apache.hadoop.security.Groups(new Configuration());
+      try {
+        List<String> g = hGroups.getGroups(System.getProperty("user.name"));
+        if (g != null && g.size() > 0) {
+          group = g.get(0);
+        }
+      } catch (NullPointerException npe) {
+        // if user/group doesn't exist on test box
+      }
+    }
+    return group;
+  }
+
+  static SolrRequest getProxyRequest(String user, String doAs) {
+    return new CollectionAdminRequest.List() {
+      @Override
+      public SolrParams getParams() {
+        ModifiableSolrParams params = new ModifiableSolrParams(super.getParams());
+        params.set(PseudoAuthenticator.USER_NAME, user);
+        params.set(KerberosPlugin.IMPERSONATOR_DO_AS_HTTP_PARAM, doAs);
+        return params;
+      }
+    };
+  }
+
+  static String getExpectedGroupExMsg(String user, String doAs) {
+    return "User: " + user + " is not allowed to impersonate " + doAs;
+  }
+
+  static String getExpectedHostExMsg(String user) {
+    return "Unauthorized connection for super-user: " + user;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/core/src/test/org/apache/solr/security/hadoop/ImpersonatorCollectionsHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/hadoop/ImpersonatorCollectionsHandler.java b/solr/core/src/test/org/apache/solr/security/hadoop/ImpersonatorCollectionsHandler.java
new file mode 100644
index 0000000..2a2a469
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/security/hadoop/ImpersonatorCollectionsHandler.java
@@ -0,0 +1,60 @@
+/*
+ * 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.security.hadoop;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import javax.servlet.http.HttpServletRequest;
+
+import org.apache.hadoop.security.authentication.client.PseudoAuthenticator;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.handler.admin.CollectionsHandler;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.security.HadoopAuthPlugin;
+import org.apache.solr.security.KerberosPlugin;
+import org.junit.Assert;
+
+/**
+ * This class extends {@linkplain CollectionsHandler} and implements extra validations
+ * for verifying proxy users support in {@linkplain HadoopAuthPlugin}
+ */
+public class ImpersonatorCollectionsHandler extends CollectionsHandler {
+  static AtomicBoolean called = new AtomicBoolean(false);
+
+  public ImpersonatorCollectionsHandler() {
+    super();
+  }
+
+  public ImpersonatorCollectionsHandler(final CoreContainer coreContainer) {
+    super(coreContainer);
+  }
+
+  @Override
+  public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
+    called.set(true);
+    super.handleRequestBody(req, rsp);
+    String doAs = req.getParams().get(KerberosPlugin.IMPERSONATOR_DO_AS_HTTP_PARAM);
+    if (doAs != null) {
+      HttpServletRequest httpRequest = (HttpServletRequest)req.getContext().get("httpRequest");
+      Assert.assertNotNull(httpRequest);
+      String user = req.getParams().get(PseudoAuthenticator.USER_NAME);
+      Assert.assertNotNull(user);
+      Assert.assertEquals(user, httpRequest.getAttribute(KerberosPlugin.IMPERSONATOR_USER_NAME));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/core/src/test/org/apache/solr/security/hadoop/TestDelegationWithHadoopAuth.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/hadoop/TestDelegationWithHadoopAuth.java b/solr/core/src/test/org/apache/solr/security/hadoop/TestDelegationWithHadoopAuth.java
new file mode 100644
index 0000000..37c6b8c
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/security/hadoop/TestDelegationWithHadoopAuth.java
@@ -0,0 +1,400 @@
+/*
+ * 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.security.hadoop;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.hadoop.security.authentication.client.PseudoAuthenticator;
+import org.apache.hadoop.util.Time;
+import org.apache.http.HttpStatus;
+import org.apache.lucene.util.Constants;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.impl.LBHttpSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.DelegationTokenRequest;
+import org.apache.solr.client.solrj.response.DelegationTokenResponse;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import junit.framework.Assert;
+
+public class TestDelegationWithHadoopAuth extends SolrCloudTestCase {
+  protected static final int NUM_SERVERS = 2;
+  protected static final String USER_1 = "foo";
+  protected static final String USER_2 = "bar";
+  private static HttpSolrClient primarySolrClient, secondarySolrClient;
+
+  @BeforeClass
+  public static void setupClass() throws Exception {
+    assumeFalse("Hadoop does not work on Windows", Constants.WINDOWS);
+
+    configureCluster(NUM_SERVERS)// nodes
+        .withSecurityJson(TEST_PATH().resolve("security").resolve("hadoop_simple_auth_with_delegation.json"))
+        .configure();
+
+    JettySolrRunner runnerPrimary = cluster.getJettySolrRunners().get(0);
+    primarySolrClient =
+        new HttpSolrClient.Builder(runnerPrimary.getBaseUrl().toString())
+            .build();
+    JettySolrRunner runnerSecondary = cluster.getJettySolrRunners().get(1);
+    secondarySolrClient =
+        new HttpSolrClient.Builder(runnerSecondary.getBaseUrl().toString())
+            .build();
+  }
+
+  @AfterClass
+  public static void tearDownClass() throws Exception {
+    if (primarySolrClient != null) {
+      primarySolrClient.close();
+      primarySolrClient = null;
+    }
+
+    if (secondarySolrClient != null) {
+      secondarySolrClient.close();
+      secondarySolrClient = null;
+    }
+  }
+
+  private String getDelegationToken(final String renewer, final String user, HttpSolrClient solrClient) throws Exception {
+    DelegationTokenRequest.Get get = new DelegationTokenRequest.Get(renewer) {
+      @Override
+      public SolrParams getParams() {
+        ModifiableSolrParams params = new ModifiableSolrParams(super.getParams());
+        params.set(PseudoAuthenticator.USER_NAME, user);
+        return params;
+      }
+    };
+    DelegationTokenResponse.Get getResponse = get.process(solrClient);
+    return getResponse.getDelegationToken();
+  }
+
+  private long renewDelegationToken(final String token, final int expectedStatusCode,
+      final String user, HttpSolrClient client) throws Exception {
+    DelegationTokenRequest.Renew renew = new DelegationTokenRequest.Renew(token) {
+      @Override
+      public SolrParams getParams() {
+        ModifiableSolrParams params = new ModifiableSolrParams(super.getParams());
+        params.set(PseudoAuthenticator.USER_NAME, user);
+        return params;
+      }
+
+      @Override
+      public Set<String> getQueryParams() {
+        Set<String> queryParams = super.getQueryParams();
+        queryParams.add(PseudoAuthenticator.USER_NAME);
+        return queryParams;
+      }
+    };
+    try {
+      DelegationTokenResponse.Renew renewResponse = renew.process(client);
+      assertEquals(HttpStatus.SC_OK, expectedStatusCode);
+      return renewResponse.getExpirationTime();
+    } catch (HttpSolrClient.RemoteSolrException ex) {
+      assertEquals(expectedStatusCode, ex.code());
+      return -1;
+    }
+  }
+
+  private void cancelDelegationToken(String token, int expectedStatusCode, HttpSolrClient client)
+  throws Exception {
+    DelegationTokenRequest.Cancel cancel = new DelegationTokenRequest.Cancel(token);
+    try {
+      cancel.process(client);
+      assertEquals(HttpStatus.SC_OK, expectedStatusCode);
+    } catch (HttpSolrClient.RemoteSolrException ex) {
+      assertEquals(expectedStatusCode, ex.code());
+    }
+  }
+
+  private void doSolrRequest(String token, int expectedStatusCode, HttpSolrClient client)
+  throws Exception {
+    doSolrRequest(token, expectedStatusCode, client, 1);
+  }
+
+  private void doSolrRequest(String token, int expectedStatusCode, HttpSolrClient client, int trials)
+  throws Exception {
+    int lastStatusCode = 0;
+    for (int i = 0; i < trials; ++i) {
+      lastStatusCode = getStatusCode(token, null, null, client);
+      if (lastStatusCode == expectedStatusCode) {
+        return;
+      }
+      Thread.sleep(1000);
+    }
+    assertEquals("Did not receieve excepted status code", expectedStatusCode, lastStatusCode);
+  }
+
+  private SolrRequest getAdminRequest(final SolrParams params) {
+    return new CollectionAdminRequest.List() {
+      @Override
+      public SolrParams getParams() {
+        ModifiableSolrParams p = new ModifiableSolrParams(super.getParams());
+        p.add(params);
+        return p;
+      }
+    };
+  }
+
+  private int getStatusCode(String token, final String user, final String op, HttpSolrClient client)
+  throws Exception {
+    SolrClient delegationTokenClient;
+    if (random().nextBoolean()) delegationTokenClient = new HttpSolrClient.Builder(client.getBaseURL().toString())
+        .withKerberosDelegationToken(token)
+        .withResponseParser(client.getParser())
+        .build();
+    else delegationTokenClient = new CloudSolrClient.Builder()
+        .withZkHost((cluster.getZkServer().getZkAddress()))
+        .withLBHttpSolrClientBuilder(new LBHttpSolrClient.Builder()
+            .withResponseParser(client.getParser())
+            .withHttpSolrClientBuilder(
+                new HttpSolrClient.Builder()
+                    .withKerberosDelegationToken(token)
+            ))
+        .build();
+    try {
+      ModifiableSolrParams p = new ModifiableSolrParams();
+      if (user != null) p.set(PseudoAuthenticator.USER_NAME, user);
+      if (op != null) p.set("op", op);
+      SolrRequest req = getAdminRequest(p);
+      if (user != null || op != null) {
+        Set<String> queryParams = new HashSet<>();
+        if (user != null) queryParams.add(PseudoAuthenticator.USER_NAME);
+        if (op != null) queryParams.add("op");
+        req.setQueryParams(queryParams);
+      }
+      try {
+        delegationTokenClient.request(req, null);
+        return HttpStatus.SC_OK;
+      } catch (HttpSolrClient.RemoteSolrException re) {
+        return re.code();
+      }
+    } finally {
+      delegationTokenClient.close();
+    }
+  }
+
+  private void doSolrRequest(SolrClient client, SolrRequest request,
+      int expectedStatusCode) throws Exception {
+    try {
+      client.request(request);
+      assertEquals(HttpStatus.SC_OK, expectedStatusCode);
+    } catch (HttpSolrClient.RemoteSolrException ex) {
+      assertEquals(expectedStatusCode, ex.code());
+    }
+  }
+
+  private void verifyTokenValid(String token) throws Exception {
+     // pass with token
+    doSolrRequest(token, HttpStatus.SC_OK, primarySolrClient);
+
+    // fail without token
+    doSolrRequest(null, ErrorCode.UNAUTHORIZED.code, primarySolrClient);
+
+    // pass with token on other server
+    doSolrRequest(token, HttpStatus.SC_OK, secondarySolrClient);
+
+    // fail without token on other server
+    doSolrRequest(null, ErrorCode.UNAUTHORIZED.code, secondarySolrClient);
+  }
+
+  /**
+   * Test basic Delegation Token get/verify
+   */
+  @Test
+  public void testDelegationTokenVerify() throws Exception {
+    // Get token
+    String token = getDelegationToken(null, USER_1, primarySolrClient);
+    assertNotNull(token);
+    verifyTokenValid(token);
+  }
+
+  private void verifyTokenCancelled(String token, HttpSolrClient client) throws Exception {
+    // fail with token on both servers.  If cancelToOtherURL is true,
+    // the request went to other url, so FORBIDDEN should be returned immediately.
+    // The cancelled token may take awhile to propogate to the standard url (via ZK).
+    // This is of course the opposite if cancelToOtherURL is false.
+    doSolrRequest(token, ErrorCode.FORBIDDEN.code, client, 10);
+
+    // fail without token on both servers
+    doSolrRequest(null, ErrorCode.UNAUTHORIZED.code, primarySolrClient);
+    doSolrRequest(null, ErrorCode.UNAUTHORIZED.code, secondarySolrClient);
+  }
+
+  @Test
+  public void testDelegationTokenCancel() throws Exception {
+    {
+      // Get token
+      String token = getDelegationToken(null, USER_1, primarySolrClient);
+      assertNotNull(token);
+
+      // cancel token, note don't need to be authenticated to cancel (no user specified)
+      cancelDelegationToken(token, HttpStatus.SC_OK, primarySolrClient);
+      verifyTokenCancelled(token, primarySolrClient);
+    }
+
+    {
+      // cancel token on different server from where we got it
+      String token = getDelegationToken(null, USER_1, primarySolrClient);
+      assertNotNull(token);
+
+      cancelDelegationToken(token, HttpStatus.SC_OK, secondarySolrClient);
+      verifyTokenCancelled(token, secondarySolrClient);
+    }
+  }
+
+  @Test
+  public void testDelegationTokenCancelFail() throws Exception {
+    // cancel a bogus token
+    cancelDelegationToken("BOGUS", ErrorCode.NOT_FOUND.code, primarySolrClient);
+
+    {
+      // cancel twice, first on same server
+      String token = getDelegationToken(null, USER_1, primarySolrClient);
+      assertNotNull(token);
+      cancelDelegationToken(token, HttpStatus.SC_OK, primarySolrClient);
+      cancelDelegationToken(token, ErrorCode.NOT_FOUND.code, secondarySolrClient);
+      cancelDelegationToken(token, ErrorCode.NOT_FOUND.code, primarySolrClient);
+    }
+
+    {
+      // cancel twice, first on other server
+      String token = getDelegationToken(null, USER_1, primarySolrClient);
+      assertNotNull(token);
+      cancelDelegationToken(token, HttpStatus.SC_OK, secondarySolrClient);
+      cancelDelegationToken(token, ErrorCode.NOT_FOUND.code, secondarySolrClient);
+      cancelDelegationToken(token, ErrorCode.NOT_FOUND.code, primarySolrClient);
+    }
+  }
+
+  private void verifyDelegationTokenRenew(String renewer, String user)
+  throws Exception {
+    {
+      // renew on same server
+      String token = getDelegationToken(renewer, user, primarySolrClient);
+      assertNotNull(token);
+      long now = Time.now();
+      assertTrue(renewDelegationToken(token, HttpStatus.SC_OK, user, primarySolrClient) > now);
+      verifyTokenValid(token);
+    }
+
+    {
+      // renew on different server
+      String token = getDelegationToken(renewer, user, primarySolrClient);
+      assertNotNull(token);
+      long now = Time.now();
+      assertTrue(renewDelegationToken(token, HttpStatus.SC_OK, user, secondarySolrClient) > now);
+      verifyTokenValid(token);
+    }
+  }
+
+  @Test
+  public void testDelegationTokenRenew() throws Exception {
+    // test with specifying renewer
+    verifyDelegationTokenRenew(USER_1, USER_1);
+
+    // test without specifying renewer
+    verifyDelegationTokenRenew(null, USER_1);
+  }
+
+  @Test
+  public void testDelegationTokenRenewFail() throws Exception {
+    // don't set renewer and try to renew as an a different user
+    String token = getDelegationToken(null, USER_1, primarySolrClient);
+    assertNotNull(token);
+    renewDelegationToken(token, ErrorCode.FORBIDDEN.code, USER_2, primarySolrClient);
+    renewDelegationToken(token, ErrorCode.FORBIDDEN.code, USER_2, secondarySolrClient);
+
+    // set renewer and try to renew as different user
+    token = getDelegationToken("renewUser", USER_1, primarySolrClient);
+    assertNotNull(token);
+    renewDelegationToken(token, ErrorCode.FORBIDDEN.code, "notRenewUser", primarySolrClient);
+    renewDelegationToken(token, ErrorCode.FORBIDDEN.code, "notRenewUser", secondarySolrClient);
+  }
+
+  /**
+   * Test that a non-delegation-token "op" http param is handled correctly
+   */
+  @Test
+  public void testDelegationOtherOp() throws Exception {
+    assertEquals(HttpStatus.SC_OK, getStatusCode(null, USER_1, "someSolrOperation", primarySolrClient));
+  }
+
+  @Test
+  public void testZNodePaths() throws Exception {
+    getDelegationToken(null, USER_1, primarySolrClient);
+    SolrZkClient zkClient = new SolrZkClient(cluster.getZkServer().getZkAddress(), 1000);
+    try {
+      assertTrue(zkClient.exists("/security/zkdtsm", true));
+      assertTrue(zkClient.exists("/security/token", true));
+    } finally {
+      zkClient.close();
+    }
+  }
+
+  /**
+   * Test HttpSolrServer's delegation token support
+   */
+  @Test
+  public void testDelegationTokenSolrClient() throws Exception {
+    // Get token
+    String token = getDelegationToken(null, USER_1, primarySolrClient);
+    assertNotNull(token);
+
+    SolrRequest request = getAdminRequest(new ModifiableSolrParams());
+
+    // test without token
+    HttpSolrClient ss =
+        new HttpSolrClient.Builder(primarySolrClient.getBaseURL().toString())
+            .withResponseParser(primarySolrClient.getParser())
+            .build();
+    try {
+      doSolrRequest(ss, request, ErrorCode.UNAUTHORIZED.code);
+    } finally {
+      ss.close();
+    }
+
+    ss = new HttpSolrClient.Builder(primarySolrClient.getBaseURL().toString())
+        .withDelegationToken(token)
+        .withResponseParser(primarySolrClient.getParser())
+        .build();
+    try {
+      // test with token via property
+      doSolrRequest(ss, request, HttpStatus.SC_OK);
+
+      // test with param -- should throw an exception
+      ModifiableSolrParams tokenParam = new ModifiableSolrParams();
+      tokenParam.set("delegation", "invalidToken");
+      try {
+        doSolrRequest(ss, getAdminRequest(tokenParam), ErrorCode.FORBIDDEN.code);
+        Assert.fail("Expected exception");
+      } catch (IllegalArgumentException ex) {}
+    } finally {
+      ss.close();
+    }
+  }
+}


[36/44] lucene-solr:jira/solr-9854: LUCENE-7590: fix test edge case

Posted by ab...@apache.org.
LUCENE-7590: fix test edge case

In case all indexed documents were deleted, the test failed to
correctly assert the number of expected missing documents.


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

Branch: refs/heads/jira/solr-9854
Commit: 4d81eee8a141c68b17c2f75cf6534fb352d94473
Parents: c8542b2
Author: Shai Erera <sh...@apache.org>
Authored: Mon Dec 19 21:51:28 2016 +0200
Committer: Shai Erera <sh...@apache.org>
Committed: Mon Dec 19 21:54:36 2016 +0200

----------------------------------------------------------------------
 .../search/TestDocValuesStatsCollector.java     | 29 +++++++++++++++-----
 1 file changed, 22 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d81eee8/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java b/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
index 3471c03..aeac785 100644
--- a/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
+++ b/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
@@ -35,6 +35,7 @@ import org.apache.lucene.document.SortedNumericDocValuesField;
 import org.apache.lucene.document.SortedSetDocValuesField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.DocValuesStats.DoubleDocValuesStats;
@@ -131,7 +132,8 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
 
         int expCount = (int) Arrays.stream(docValues).filter(v -> v > 0).count();
         assertEquals(expCount, stats.count());
-        assertEquals(getZeroValues(docValues).count() - reader.numDeletedDocs(), stats.missing());
+        int numDocsWithoutField = (int) getZeroValues(docValues).count();
+        assertEquals(computeExpMissing(numDocsWithoutField, numDocs, reader), stats.missing());
         if (stats.count() > 0) {
           LongSummaryStatistics sumStats = getPositiveValues(docValues).summaryStatistics();
           assertEquals(sumStats.getMax(), stats.max().longValue());
@@ -181,7 +183,8 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
 
         int expCount = (int) Arrays.stream(docValues).filter(v -> v > 0).count();
         assertEquals(expCount, stats.count());
-        assertEquals(getZeroValues(docValues).count() - reader.numDeletedDocs(), stats.missing());
+        int numDocsWithoutField = (int) getZeroValues(docValues).count();
+        assertEquals(computeExpMissing(numDocsWithoutField, numDocs, reader), stats.missing());
         if (stats.count() > 0) {
           DoubleSummaryStatistics sumStats = getPositiveValues(docValues).summaryStatistics();
           assertEquals(sumStats.getMax(), stats.max().doubleValue(), 0.00001);
@@ -234,7 +237,8 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
         searcher.search(new MatchAllDocsQuery(), new DocValuesStatsCollector(stats));
 
         assertEquals(nonNull(docValues).count(), stats.count());
-        assertEquals(isNull(docValues).count() - reader.numDeletedDocs(), stats.missing());
+        int numDocsWithoutField = (int) isNull(docValues).count();
+        assertEquals(computeExpMissing(numDocsWithoutField, numDocs, reader), stats.missing());
         if (stats.count() > 0) {
           LongSummaryStatistics sumStats = filterAndFlatValues(docValues, (v) -> v != null).summaryStatistics();
           assertEquals(sumStats.getMax(), stats.max().longValue());
@@ -288,7 +292,8 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
         searcher.search(new MatchAllDocsQuery(), new DocValuesStatsCollector(stats));
 
         assertEquals(nonNull(docValues).count(), stats.count());
-        assertEquals(isNull(docValues).count() - reader.numDeletedDocs(), stats.missing());
+        int numDocsWithoutField = (int) isNull(docValues).count();
+        assertEquals(computeExpMissing(numDocsWithoutField, numDocs, reader), stats.missing());
         if (stats.count() > 0) {
           DoubleSummaryStatistics sumStats = filterAndFlatValues(docValues, (v) -> v != null).summaryStatistics();
           assertEquals(sumStats.getMax(), stats.max().longValue(), 0.00001);
@@ -338,7 +343,8 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
 
         int expCount = (int) nonNull(docValues).count();
         assertEquals(expCount, stats.count());
-        assertEquals(isNull(docValues).count() - reader.numDeletedDocs(), stats.missing());
+        int numDocsWithoutField = (int) isNull(docValues).count();
+        assertEquals(computeExpMissing(numDocsWithoutField, numDocs, reader), stats.missing());
         if (stats.count() > 0) {
           assertEquals(nonNull(docValues).min(BytesRef::compareTo).get(), stats.min());
           assertEquals(nonNull(docValues).max(BytesRef::compareTo).get(), stats.max());
@@ -381,11 +387,13 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
       try (DirectoryReader reader = DirectoryReader.open(indexWriter)) {
         IndexSearcher searcher = new IndexSearcher(reader);
         SortedSetDocValuesStats stats = new SortedSetDocValuesStats(field);
-        searcher.search(new MatchAllDocsQuery(), new DocValuesStatsCollector(stats));
+        TotalHitCountCollector totalHitCount = new TotalHitCountCollector();
+        searcher.search(new MatchAllDocsQuery(), MultiCollector.wrap(totalHitCount, new DocValuesStatsCollector(stats)));
 
         int expCount = (int) nonNull(docValues).count();
         assertEquals(expCount, stats.count());
-        assertEquals(isNull(docValues).count() - reader.numDeletedDocs(), stats.missing());
+        int numDocsWithoutField = (int) isNull(docValues).count();
+        assertEquals(computeExpMissing(numDocsWithoutField, numDocs, reader), stats.missing());
         if (stats.count() > 0) {
           assertEquals(nonNull(docValues).flatMap(Arrays::stream).min(BytesRef::compareTo).get(), stats.min());
           assertEquals(nonNull(docValues).flatMap(Arrays::stream).max(BytesRef::compareTo).get(), stats.max());
@@ -445,4 +453,11 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
   private static <T> Stream<T> filterValues(T[] values, Predicate<? super T> p) {
     return Arrays.stream(values).filter(p);
   }
+
+  private static int computeExpMissing(int numDocsWithoutField, int numIndexedDocs, IndexReader reader) {
+    // The number of missing documents equals the number of docs without the field (not indexed with it, or were
+    // deleted). However, in case we deleted all documents in a segment before the reader was opened, there will be
+    // a mismatch between numDocs (how many we indexed) to reader.maxDoc(), so compensate for that.
+    return numDocsWithoutField - reader.numDeletedDocs() - (numIndexedDocs - reader.maxDoc());
+  }
 }


[24/44] lucene-solr:jira/solr-9854: LUCENE-7590: add Sorted(Set)DocValuesStats

Posted by ab...@apache.org.
LUCENE-7590: add Sorted(Set)DocValuesStats


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

Branch: refs/heads/jira/solr-9854
Commit: 23206caabd09310cb23a2b5302ce41af62b5c9cc
Parents: 944b8e0
Author: Shai Erera <sh...@apache.org>
Authored: Sun Dec 18 12:20:42 2016 +0200
Committer: Shai Erera <sh...@apache.org>
Committed: Sun Dec 18 21:50:03 2016 +0200

----------------------------------------------------------------------
 .../apache/lucene/search/DocValuesStats.java    |  83 ++++++++++++
 .../search/TestDocValuesStatsCollector.java     | 127 +++++++++++++++++--
 2 files changed, 196 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/23206caa/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java b/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
index 9dd97a6..b6449cc 100644
--- a/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
+++ b/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
@@ -20,7 +20,11 @@ import java.io.IOException;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.SortedNumericDocValues;
+import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
 
 /** Holds statistics for a DocValues field. */
 public abstract class DocValuesStats<T> {
@@ -323,4 +327,83 @@ public abstract class DocValuesStats<T> {
     }
   }
 
+  private static BytesRef copyFrom(BytesRef src, BytesRef dest) {
+    if (dest == null) {
+      return BytesRef.deepCopyOf(src);
+    }
+
+    dest.bytes = ArrayUtil.grow(dest.bytes, src.length);
+    System.arraycopy(src.bytes, src.offset, dest.bytes, 0, src.length);
+    dest.offset = 0;
+    dest.length = src.length;
+    return dest;
+  }
+
+  /** Holds statistics for a sorted DocValues field. */
+  public static class SortedDocValuesStats extends DocValuesStats<BytesRef> {
+
+    protected SortedDocValues sdv;
+
+    protected SortedDocValuesStats(String field) {
+      super(field, null, null);
+    }
+
+    @Override
+    protected final boolean init(LeafReaderContext context) throws IOException {
+      sdv = context.reader().getSortedDocValues(field);
+      return sdv != null;
+    }
+
+    @Override
+    protected final boolean hasValue(int doc) throws IOException {
+      return sdv.advanceExact(doc);
+    }
+
+    @Override
+    protected void doAccumulate(int count) throws IOException {
+      BytesRef val = sdv.binaryValue();
+      if (max == null || val.compareTo(max) > 0) {
+        max = copyFrom(val, max);
+      }
+      if (min == null || val.compareTo(min) < 0) {
+        min = copyFrom(val, min);
+      }
+    }
+  }
+
+  /** Holds statistics for a sorted-set DocValues field. */
+  public static class SortedSetDocValuesStats extends DocValuesStats<BytesRef> {
+
+    protected SortedSetDocValues ssdv;
+
+    protected SortedSetDocValuesStats(String field) {
+      super(field, null, null);
+    }
+
+    @Override
+    protected final boolean init(LeafReaderContext context) throws IOException {
+      ssdv = context.reader().getSortedSetDocValues(field);
+      return ssdv != null;
+    }
+
+    @Override
+    protected final boolean hasValue(int doc) throws IOException {
+      return ssdv.advanceExact(doc);
+    }
+
+    @Override
+    protected void doAccumulate(int count) throws IOException {
+      long ord;
+      while ((ord = ssdv.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
+        BytesRef val = ssdv.lookupOrd(ord);
+        if (max == null || val.compareTo(max) > 0) {
+          max = copyFrom(val, max);
+        }
+        if (min == null || val.compareTo(min) < 0) {
+          min = copyFrom(val, min);
+        }
+      }
+    }
+  }
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/23206caa/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java b/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
index 5fa4b04..3471c03 100644
--- a/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
+++ b/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
@@ -20,6 +20,7 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.DoubleSummaryStatistics;
 import java.util.LongSummaryStatistics;
+import java.util.Objects;
 import java.util.function.Predicate;
 import java.util.stream.DoubleStream;
 import java.util.stream.LongStream;
@@ -29,16 +30,21 @@ import org.apache.lucene.document.Document;
 import org.apache.lucene.document.DoubleDocValuesField;
 import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.document.SortedNumericDocValuesField;
+import org.apache.lucene.document.SortedSetDocValuesField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.DocValuesStats.DoubleDocValuesStats;
 import org.apache.lucene.search.DocValuesStats.LongDocValuesStats;
+import org.apache.lucene.search.DocValuesStats.SortedDocValuesStats;
 import org.apache.lucene.search.DocValuesStats.SortedDoubleDocValuesStats;
 import org.apache.lucene.search.DocValuesStats.SortedLongDocValuesStats;
+import org.apache.lucene.search.DocValuesStats.SortedSetDocValuesStats;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 
@@ -227,8 +233,8 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
         SortedLongDocValuesStats stats = new SortedLongDocValuesStats(field);
         searcher.search(new MatchAllDocsQuery(), new DocValuesStatsCollector(stats));
 
-        assertEquals(filterValues(docValues, (v) -> v != null).count(), stats.count());
-        assertEquals(filterValues(docValues, (v) -> v == null).count() - reader.numDeletedDocs(), stats.missing());
+        assertEquals(nonNull(docValues).count(), stats.count());
+        assertEquals(isNull(docValues).count() - reader.numDeletedDocs(), stats.missing());
         if (stats.count() > 0) {
           LongSummaryStatistics sumStats = filterAndFlatValues(docValues, (v) -> v != null).summaryStatistics();
           assertEquals(sumStats.getMax(), stats.max().longValue());
@@ -281,8 +287,8 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
         SortedDoubleDocValuesStats stats = new SortedDoubleDocValuesStats(field);
         searcher.search(new MatchAllDocsQuery(), new DocValuesStatsCollector(stats));
 
-        assertEquals(filterValues(docValues, (v) -> v != null).count(), stats.count());
-        assertEquals(filterValues(docValues, (v) -> v == null).count() - reader.numDeletedDocs(), stats.missing());
+        assertEquals(nonNull(docValues).count(), stats.count());
+        assertEquals(isNull(docValues).count() - reader.numDeletedDocs(), stats.missing());
         if (stats.count() > 0) {
           DoubleSummaryStatistics sumStats = filterAndFlatValues(docValues, (v) -> v != null).summaryStatistics();
           assertEquals(sumStats.getMax(), stats.max().longValue(), 0.00001);
@@ -298,6 +304,96 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
     }
   }
 
+  public void testDocsWithSortedValues() throws IOException {
+    try (Directory dir = newDirectory();
+        IndexWriter indexWriter = new IndexWriter(dir, newIndexWriterConfig())) {
+      String field = "sorted";
+      int numDocs = TestUtil.nextInt(random(), 1, 100);
+      BytesRef[] docValues = new BytesRef[numDocs];
+      for (int i = 0; i < numDocs; i++) {
+        Document doc = new Document();
+        if (random().nextBoolean()) { // not all documents have a value
+          BytesRef val = TestUtil.randomBinaryTerm(random());
+          doc.add(new SortedDocValuesField(field, val));
+          doc.add(new StringField("id", "doc" + i, Store.NO));
+          docValues[i] = val;
+        }
+        indexWriter.addDocument(doc);
+      }
+
+      // 20% of cases delete some docs
+      if (random().nextDouble() < 0.2) {
+        for (int i = 0; i < numDocs; i++) {
+          if (random().nextBoolean()) {
+            indexWriter.deleteDocuments(new Term("id", "doc" + i));
+            docValues[i] = null;
+          }
+        }
+      }
+
+      try (DirectoryReader reader = DirectoryReader.open(indexWriter)) {
+        IndexSearcher searcher = new IndexSearcher(reader);
+        SortedDocValuesStats stats = new SortedDocValuesStats(field);
+        searcher.search(new MatchAllDocsQuery(), new DocValuesStatsCollector(stats));
+
+        int expCount = (int) nonNull(docValues).count();
+        assertEquals(expCount, stats.count());
+        assertEquals(isNull(docValues).count() - reader.numDeletedDocs(), stats.missing());
+        if (stats.count() > 0) {
+          assertEquals(nonNull(docValues).min(BytesRef::compareTo).get(), stats.min());
+          assertEquals(nonNull(docValues).max(BytesRef::compareTo).get(), stats.max());
+        }
+      }
+    }
+  }
+
+  public void testDocsWithSortedSetValues() throws IOException {
+    try (Directory dir = newDirectory();
+        IndexWriter indexWriter = new IndexWriter(dir, newIndexWriterConfig())) {
+      String field = "sorted";
+      int numDocs = TestUtil.nextInt(random(), 1, 100);
+      BytesRef[][] docValues = new BytesRef[numDocs][];
+      for (int i = 0; i < numDocs; i++) {
+        Document doc = new Document();
+        if (random().nextBoolean()) { // not all documents have a value
+          int numValues = TestUtil.nextInt(random(), 1, 5);
+          docValues[i] = new BytesRef[numValues];
+          for (int j = 0; j < numValues; j++) {
+            BytesRef val = TestUtil.randomBinaryTerm(random());
+            doc.add(new SortedSetDocValuesField(field, val));
+            docValues[i][j] = val;
+          }
+          doc.add(new StringField("id", "doc" + i, Store.NO));
+        }
+        indexWriter.addDocument(doc);
+      }
+
+      // 20% of cases delete some docs
+      if (random().nextDouble() < 0.2) {
+        for (int i = 0; i < numDocs; i++) {
+          if (random().nextBoolean()) {
+            indexWriter.deleteDocuments(new Term("id", "doc" + i));
+            docValues[i] = null;
+          }
+        }
+      }
+
+      try (DirectoryReader reader = DirectoryReader.open(indexWriter)) {
+        IndexSearcher searcher = new IndexSearcher(reader);
+        SortedSetDocValuesStats stats = new SortedSetDocValuesStats(field);
+        searcher.search(new MatchAllDocsQuery(), new DocValuesStatsCollector(stats));
+
+        int expCount = (int) nonNull(docValues).count();
+        assertEquals(expCount, stats.count());
+        assertEquals(isNull(docValues).count() - reader.numDeletedDocs(), stats.missing());
+        if (stats.count() > 0) {
+          assertEquals(nonNull(docValues).flatMap(Arrays::stream).min(BytesRef::compareTo).get(), stats.min());
+          assertEquals(nonNull(docValues).flatMap(Arrays::stream).max(BytesRef::compareTo).get(), stats.max());
+        }
+      }
+    }
+  }
+
   private static LongStream getPositiveValues(long[] values) {
     return Arrays.stream(values).filter(v -> v > 0);
   }
@@ -314,14 +410,6 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
     return Arrays.stream(values).filter(v -> v == 0);
   }
 
-  private static Stream<long[]> filterValues(long[][] values, Predicate<? super long[]> p) {
-    return Arrays.stream(values).filter(p);
-  }
-
-  private static Stream<double[]> filterValues(double[][] values, Predicate<? super double[]> p) {
-    return Arrays.stream(values).filter(p);
-  }
-
   private static double computeVariance(long[] values, double mean, int count) {
     return getPositiveValues(values).mapToDouble(v -> (v - mean) * (v-mean)).sum() / count;
   }
@@ -331,11 +419,11 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
   }
 
   private static LongStream filterAndFlatValues(long[][] values, Predicate<? super long[]> p) {
-    return filterValues(values, (v) -> v != null).flatMapToLong(Arrays::stream);
+    return nonNull(values).flatMapToLong(Arrays::stream);
   }
 
   private static DoubleStream filterAndFlatValues(double[][] values, Predicate<? super double[]> p) {
-    return filterValues(values, (v) -> v != null).flatMapToDouble(Arrays::stream);
+    return nonNull(values).flatMapToDouble(Arrays::stream);
   }
 
   private static double computeVariance(LongStream values, double mean, int count) {
@@ -346,4 +434,15 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
     return values.map(v -> (v - mean) * (v-mean)).sum() / count;
   }
 
+  private static <T> Stream<T> nonNull(T[] values) {
+    return filterValues(values, Objects::nonNull);
+  }
+
+  private static <T> Stream<T> isNull(T[] values) {
+    return filterValues(values, Objects::isNull);
+  }
+
+  private static <T> Stream<T> filterValues(T[] values, Predicate<? super T> p) {
+    return Arrays.stream(values).filter(p);
+  }
 }


[32/44] lucene-solr:jira/solr-9854: LUCENE-7586: fail precommit on varargsArgumentNeedCast

Posted by ab...@apache.org.
LUCENE-7586: fail precommit on varargsArgumentNeedCast


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

Branch: refs/heads/jira/solr-9854
Commit: b22451f45ef791ef5985f95327e0b3c382e4c79a
Parents: 2c4c504
Author: Christine Poerschke <cp...@apache.org>
Authored: Mon Dec 19 12:51:10 2016 +0000
Committer: Christine Poerschke <cp...@apache.org>
Committed: Mon Dec 19 16:14:51 2016 +0000

----------------------------------------------------------------------
 dev-tools/eclipse/dot.settings/org.eclipse.jdt.core.prefs | 1 +
 lucene/tools/javadoc/ecj.javadocs.prefs                   | 2 +-
 2 files changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b22451f4/dev-tools/eclipse/dot.settings/org.eclipse.jdt.core.prefs
----------------------------------------------------------------------
diff --git a/dev-tools/eclipse/dot.settings/org.eclipse.jdt.core.prefs b/dev-tools/eclipse/dot.settings/org.eclipse.jdt.core.prefs
index 0dea4ea..6f6533a 100644
--- a/dev-tools/eclipse/dot.settings/org.eclipse.jdt.core.prefs
+++ b/dev-tools/eclipse/dot.settings/org.eclipse.jdt.core.prefs
@@ -307,3 +307,4 @@ org.eclipse.jdt.core.formatter.wrap_outer_expressions_when_nested=true
 org.eclipse.jdt.core.compiler.problem.comparingIdentical=error
 org.eclipse.jdt.core.compiler.problem.noEffectAssignment=error
 org.eclipse.jdt.core.compiler.problem.unusedImport=error
+org.eclipse.jdt.core.compiler.problem.varargsArgumentNeedCast=error

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b22451f4/lucene/tools/javadoc/ecj.javadocs.prefs
----------------------------------------------------------------------
diff --git a/lucene/tools/javadoc/ecj.javadocs.prefs b/lucene/tools/javadoc/ecj.javadocs.prefs
index c81d8e3..61971ec 100644
--- a/lucene/tools/javadoc/ecj.javadocs.prefs
+++ b/lucene/tools/javadoc/ecj.javadocs.prefs
@@ -92,5 +92,5 @@ org.eclipse.jdt.core.compiler.problem.unusedParameterWhenImplementingAbstract=di
 org.eclipse.jdt.core.compiler.problem.unusedParameterWhenOverridingConcrete=disabled
 org.eclipse.jdt.core.compiler.problem.unusedPrivateMember=ignore
 org.eclipse.jdt.core.compiler.problem.unusedWarningToken=ignore
-org.eclipse.jdt.core.compiler.problem.varargsArgumentNeedCast=ignore
+org.eclipse.jdt.core.compiler.problem.varargsArgumentNeedCast=error
 org.eclipse.jdt.core.compiler.source=1.8


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

Posted by ab...@apache.org.
SOLR-9812: Added a new /admin/metrics API to return all metrics collected by Solr via API


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

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

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


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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aa9b02bb/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java
new file mode 100644
index 0000000..1adc480
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.handler.admin;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.Meter;
+import com.codahale.metrics.MetricFilter;
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.Timer;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.handler.RequestHandlerBase;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.security.AuthorizationContext;
+import org.apache.solr.security.PermissionNameProvider;
+import org.apache.solr.util.stats.MetricUtils;
+
+/**
+ * Request handler to return metrics
+ */
+public class MetricsHandler extends RequestHandlerBase implements PermissionNameProvider {
+  final CoreContainer container;
+
+  public MetricsHandler() {
+    this.container = null;
+  }
+
+  public MetricsHandler(CoreContainer container) {
+    this.container = container;
+  }
+
+  @Override
+  public Name getPermissionName(AuthorizationContext request) {
+    return Name.METRICS_READ_PERM;
+  }
+
+  @Override
+  public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
+    if (container == null) {
+      throw new SolrException(SolrException.ErrorCode.INVALID_STATE, "Core container instance not initialized");
+    }
+
+    List<MetricType> metricTypes = parseMetricTypes(req);
+    List<MetricFilter> metricFilters = metricTypes.stream().map(MetricType::asMetricFilter).collect(Collectors.toList());
+    List<Group> requestedGroups = parseGroups(req);
+
+    NamedList response = new NamedList();
+    for (Group group : requestedGroups) {
+      String registryName = SolrMetricManager.getRegistryName(group);
+      if (group == Group.core) {
+        // this requires special handling because of the way we create registry name for a core (deeply nested)
+        container.getAllCoreNames().forEach(s -> {
+          String coreRegistryName;
+          try (SolrCore core = container.getCore(s)) {
+            coreRegistryName = core.getMetricManager().getRegistryName();
+          }
+          MetricRegistry registry = SolrMetricManager.registry(coreRegistryName);
+          response.add(coreRegistryName, MetricUtils.toNamedList(registry, metricFilters));
+        });
+      } else {
+        MetricRegistry registry = SolrMetricManager.registry(registryName);
+        response.add(registryName, MetricUtils.toNamedList(registry, metricFilters));
+      }
+    }
+    rsp.getValues().add("metrics", response);
+  }
+
+  private List<Group> parseGroups(SolrQueryRequest req) {
+    String[] groupStr = req.getParams().getParams("group");
+    List<String> groups = Collections.emptyList();
+    if (groupStr != null && groupStr.length > 0) {
+      groups = new ArrayList<>();
+      for (String g : groupStr) {
+        groups.addAll(StrUtils.splitSmart(g, ','));
+      }
+    }
+
+    List<Group> requestedGroups = Arrays.asList(Group.values()); // by default we return all groups
+    try {
+      if (groups.size() > 0 && !groups.contains("all")) {
+        requestedGroups = groups.stream().map(String::trim).map(Group::valueOf).collect(Collectors.toList());
+      }
+    } catch (IllegalArgumentException e) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Invalid group in: " + groups + " specified. Must be one of (all, jvm, jetty, http, node, core)", e);
+    }
+    return requestedGroups;
+  }
+
+  private List<MetricType> parseMetricTypes(SolrQueryRequest req) {
+    String[] typeStr = req.getParams().getParams("type");
+    List<String> types = Collections.emptyList();
+    if (typeStr != null && typeStr.length > 0)  {
+      types = new ArrayList<>();
+      for (String type : typeStr) {
+        types.addAll(StrUtils.splitSmart(type, ','));
+      }
+    }
+
+    List<MetricType> metricTypes = Collections.singletonList(MetricType.all); // include all metrics by default
+    try {
+      if (types.size() > 0) {
+        metricTypes = types.stream().map(String::trim).map(MetricType::valueOf).collect(Collectors.toList());
+      }
+    } catch (IllegalArgumentException e) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Invalid metric type in: " + types + " specified. Must be one of (all, meter, timer, histogram, counter, gauge)", e);
+    }
+    return metricTypes;
+  }
+
+  @Override
+  public String getDescription() {
+    return "A handler to return all the metrics gathered by Solr";
+  }
+
+  enum MetricType {
+    histogram(Histogram.class),
+    meter(Meter.class),
+    timer(Timer.class),
+    counter(Counter.class),
+    gauge(Gauge.class),
+    all(null);
+
+    private final Class klass;
+
+    MetricType(Class klass) {
+      this.klass = klass;
+    }
+
+    public MetricFilter asMetricFilter() {
+      return (name, metric) -> klass == null || klass.isInstance(metric);
+    }
+  }
+}

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

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

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

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aa9b02bb/solr/core/src/java/org/apache/solr/servlet/SolrMetricsServletContextListener.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/SolrMetricsServletContextListener.java b/solr/core/src/java/org/apache/solr/servlet/SolrMetricsServletContextListener.java
deleted file mode 100644
index 041ef11..0000000
--- a/solr/core/src/java/org/apache/solr/servlet/SolrMetricsServletContextListener.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.solr.servlet;
-
-import com.codahale.metrics.MetricRegistry;
-import com.codahale.metrics.SharedMetricRegistries;
-import com.codahale.metrics.servlets.MetricsServlet;
-
-public class SolrMetricsServletContextListener extends MetricsServlet.ContextListener {
-  @Override
-  protected MetricRegistry getMetricRegistry() {
-    return SharedMetricRegistries.getOrCreate("solr.jetty");
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aa9b02bb/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java b/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java
new file mode 100644
index 0000000..62f5776
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.util.stats;
+
+import java.util.List;
+import java.util.Map;
+import java.util.SortedSet;
+import java.util.concurrent.TimeUnit;
+
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.Meter;
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricFilter;
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.Snapshot;
+import com.codahale.metrics.Timer;
+import org.apache.solr.common.util.NamedList;
+
+/**
+ * Metrics specific utility functions.
+ */
+public class MetricUtils {
+
+  /**
+   * Adds metrics from a Timer to a NamedList, using well-known names.
+   * @param lst The NamedList to add the metrics data to
+   * @param timer The Timer to extract the metrics from
+   */
+  public static void addMetrics(NamedList<Object> lst, Timer timer) {
+    Snapshot snapshot = timer.getSnapshot();
+    lst.add("avgRequestsPerSecond", timer.getMeanRate());
+    lst.add("5minRateRequestsPerSecond", timer.getFiveMinuteRate());
+    lst.add("15minRateRequestsPerSecond", timer.getFifteenMinuteRate());
+    lst.add("avgTimePerRequest", nsToMs(snapshot.getMean()));
+    lst.add("medianRequestTime", nsToMs(snapshot.getMedian()));
+    lst.add("75thPcRequestTime", nsToMs(snapshot.get75thPercentile()));
+    lst.add("95thPcRequestTime", nsToMs(snapshot.get95thPercentile()));
+    lst.add("99thPcRequestTime", nsToMs(snapshot.get99thPercentile()));
+    lst.add("999thPcRequestTime", nsToMs(snapshot.get999thPercentile()));
+  }
+
+  /**
+   * Converts a double representing nanoseconds to a double representing milliseconds.
+   *
+   * @param ns the amount of time in nanoseconds
+   * @return the amount of time in milliseconds
+   */
+  static double nsToMs(double ns) {
+    return ns / TimeUnit.MILLISECONDS.toNanos(1);
+  }
+
+  /**
+   * Returns a NamedList respresentation of the given metric registry. Only those metrics
+   * are converted to NamedList which match at least one of the given MetricFilter instances.
+   *
+   * @param registry      the {@link MetricRegistry} to be converted to NamedList
+   * @param metricFilters a list of {@link MetricFilter} instances
+   * @return a {@link NamedList}
+   */
+  public static NamedList toNamedList(MetricRegistry registry, List<MetricFilter> metricFilters) {
+    NamedList response = new NamedList();
+    Map<String, Metric> metrics = registry.getMetrics();
+    SortedSet<String> names = registry.getNames();
+    names.stream().filter(s -> metricFilters.stream().anyMatch(metricFilter -> metricFilter.matches(s, metrics.get(s)))).forEach(n -> {
+      Metric metric = metrics.get(n);
+      if (metric instanceof Counter) {
+        Counter counter = (Counter) metric;
+        response.add(n, counterToNamedList(counter));
+      } else if (metric instanceof Gauge) {
+        Gauge gauge = (Gauge) metric;
+        response.add(n, gaugeToNamedList(gauge));
+      } else if (metric instanceof Meter) {
+        Meter meter = (Meter) metric;
+        response.add(n, meterToNamedList(meter));
+      } else if (metric instanceof Timer) {
+        Timer timer = (Timer) metric;
+        response.add(n, timerToNamedList(timer));
+      } else if (metric instanceof Histogram) {
+        Histogram histogram = (Histogram) metric;
+        response.add(n, histogramToNamedList(histogram));
+      }
+    });
+    return response;
+  }
+
+  static NamedList histogramToNamedList(Histogram histogram) {
+    NamedList response = new NamedList();
+    Snapshot snapshot = histogram.getSnapshot();
+    response.add("requests", histogram.getCount());
+    response.add("minTime", nsToMs(snapshot.getMin()));
+    response.add("maxTime", nsToMs(snapshot.getMax()));
+    response.add("avgTimePerRequest", nsToMs(snapshot.getMean()));
+    response.add("medianRequestTime", nsToMs(snapshot.getMedian()));
+    response.add("75thPcRequestTime", nsToMs(snapshot.get75thPercentile()));
+    response.add("95thPcRequestTime", nsToMs(snapshot.get95thPercentile()));
+    response.add("99thPcRequestTime", nsToMs(snapshot.get99thPercentile()));
+    response.add("999thPcRequestTime", nsToMs(snapshot.get999thPercentile()));
+    return response;
+  }
+
+  static NamedList timerToNamedList(Timer timer) {
+    NamedList response = new NamedList();
+    addMetrics(response, timer);
+    return response;
+  }
+
+  static NamedList meterToNamedList(Meter meter) {
+    NamedList response = new NamedList();
+    response.add("requests", meter.getCount());
+    response.add("avgRequestsPerSecond", meter.getMeanRate());
+    response.add("1minRateRequestsPerSecond", meter.getOneMinuteRate());
+    response.add("5minRateRequestsPerSecond", meter.getFiveMinuteRate());
+    response.add("15minRateRequestsPerSecond", meter.getFifteenMinuteRate());
+    return response;
+  }
+
+  static NamedList gaugeToNamedList(Gauge gauge) {
+    NamedList response = new NamedList();
+    response.add("value", gauge.getValue());
+    return response;
+  }
+
+  static NamedList counterToNamedList(Counter counter) {
+    NamedList response = new NamedList();
+    response.add("requests", counter.getCount());
+    return response;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aa9b02bb/solr/core/src/java/org/apache/solr/util/stats/TimerUtils.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/stats/TimerUtils.java b/solr/core/src/java/org/apache/solr/util/stats/TimerUtils.java
deleted file mode 100644
index 243c1ee..0000000
--- a/solr/core/src/java/org/apache/solr/util/stats/TimerUtils.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.util.stats;
-
-import java.util.concurrent.TimeUnit;
-
-import com.codahale.metrics.Snapshot;
-import com.codahale.metrics.Timer;
-import org.apache.solr.common.util.NamedList;
-
-/**
- * Solr specific {@link Timer} utility functions.
- */
-public class TimerUtils {
-
-  /**
-   * Adds metrics from a Timer to a NamedList, using well-known names.
-   * @param lst The NamedList to add the metrics data to
-   * @param timer The Timer to extract the metrics from
-   */
-  public static void addMetrics(NamedList<Object> lst, Timer timer) {
-    Snapshot snapshot = timer.getSnapshot();
-    lst.add("avgRequestsPerSecond", timer.getMeanRate());
-    lst.add("5minRateRequestsPerSecond", timer.getFiveMinuteRate());
-    lst.add("15minRateRequestsPerSecond", timer.getFifteenMinuteRate());
-    lst.add("avgTimePerRequest", nsToMs(snapshot.getMean()));
-    lst.add("medianRequestTime", nsToMs(snapshot.getMedian()));
-    lst.add("75thPcRequestTime", nsToMs(snapshot.get75thPercentile()));
-    lst.add("95thPcRequestTime", nsToMs(snapshot.get95thPercentile()));
-    lst.add("99thPcRequestTime", nsToMs(snapshot.get99thPercentile()));
-    lst.add("999thPcRequestTime", nsToMs(snapshot.get999thPercentile()));
-  }
-
-  /**
-   * Converts a double representing nanoseconds to a double representing milliseconds.
-   *
-   * @param ns the amount of time in nanoseconds
-   * @return the amount of time in milliseconds
-   */
-  static double nsToMs(double ns) {
-    return ns / TimeUnit.MILLISECONDS.toNanos(1);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aa9b02bb/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
new file mode 100644
index 0000000..83ce74f
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.handler.admin;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.response.SolrQueryResponse;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test for {@link MetricsHandler}
+ */
+public class MetricsHandlerTest extends SolrTestCaseJ4 {
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    initCore("solrconfig.xml", "schema.xml");
+  }
+
+  @Test
+  public void test() throws Exception {
+    MetricsHandler handler = new MetricsHandler(h.getCoreContainer());
+
+    SolrQueryResponse resp = new SolrQueryResponse();
+    handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json"), resp);
+    NamedList values = resp.getValues();
+    assertNotNull(values.get("metrics"));
+    values = (NamedList) values.get("metrics");
+    System.out.println(values);
+    assertNotNull(values.get("solr.jetty"));
+    assertNotNull(values.get("solr.jvm"));
+    assertNotNull(values.get("solr.http"));
+    assertNotNull(values.get("solr.node"));
+    NamedList nl = (NamedList) values.get("solr.core.collection1");
+    assertNotNull(nl);
+    assertNotNull(nl.get("newSearcherErrors")); // counter type
+    assertNotNull(((NamedList) nl.get("newSearcherErrors")).get("requests"));
+    assertEquals(0L, ((NamedList) nl.get("newSearcherErrors")).get("requests"));
+    nl = (NamedList) values.get("solr.node");
+    assertNotNull(nl.get("QUERYHANDLER./admin/authorization.clientErrors")); // timer type
+    assertEquals(5, ((NamedList) nl.get("QUERYHANDLER./admin/authorization.clientErrors")).size());
+
+    resp = new SolrQueryResponse();
+    handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", "group", "jvm,jetty"), resp);
+    values = resp.getValues();
+    assertNotNull(values.get("metrics"));
+    values = (NamedList) values.get("metrics");
+    assertEquals(2, values.size());
+    assertNotNull(values.get("solr.jetty"));
+    assertNotNull(values.get("solr.jvm"));
+
+    resp = new SolrQueryResponse();
+    handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", "group", "jvm,jetty"), resp);
+    values = resp.getValues();
+    assertNotNull(values.get("metrics"));
+    values = (NamedList) values.get("metrics");
+    assertEquals(2, values.size());
+    assertNotNull(values.get("solr.jetty"));
+    assertNotNull(values.get("solr.jvm"));
+
+    resp = new SolrQueryResponse();
+    handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", "group", "jvm", "group", "jetty"), resp);
+    values = resp.getValues();
+    assertNotNull(values.get("metrics"));
+    values = (NamedList) values.get("metrics");
+    assertEquals(2, values.size());
+    assertNotNull(values.get("solr.jetty"));
+    assertNotNull(values.get("solr.jvm"));
+
+    resp = new SolrQueryResponse();
+    handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", "group", "node", "type", "counter"), resp);
+    values = resp.getValues();
+    assertNotNull(values.get("metrics"));
+    values = (NamedList) values.get("metrics");
+    assertEquals(1, values.size());
+    assertNotNull(values.get("solr.node"));
+    assertNull(values.get("QUERYHANDLER./admin/authorization.errors")); // this is a timer node
+  }
+}

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

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

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

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

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

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

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


[02/44] lucene-solr:jira/solr-9854: LUCENE-7590: add DocValuesStatsCollector

Posted by ab...@apache.org.
LUCENE-7590: add DocValuesStatsCollector


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

Branch: refs/heads/jira/solr-9854
Commit: ad7152ad4739a47aa2b45405ba1682b3dda18923
Parents: 9aa5b73
Author: Shai Erera <sh...@apache.org>
Authored: Sun Dec 11 12:49:50 2016 +0200
Committer: Shai Erera <sh...@apache.org>
Committed: Wed Dec 14 13:04:52 2016 +0200

----------------------------------------------------------------------
 .../apache/lucene/search/DocValuesStats.java    | 165 ++++++++++++++++++
 .../lucene/search/DocValuesStatsCollector.java  |  64 +++++++
 .../search/TestDocValuesStatsCollector.java     | 166 +++++++++++++++++++
 3 files changed, 395 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ad7152ad/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java b/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
new file mode 100644
index 0000000..fad9f97
--- /dev/null
+++ b/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
@@ -0,0 +1,165 @@
+/*
+ * 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.lucene.search;
+
+import java.io.IOException;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+
+/** Holds statistics for a DocValues field. */
+public abstract class DocValuesStats<T> {
+
+  private int missing = 0;
+  private int count = 0;
+
+  protected final String field;
+
+  protected T min;
+  protected T max;
+
+  protected DocValuesStats(String field, T initialMin, T initialMax) {
+    this.field = field;
+    this.min = initialMin;
+    this.max = initialMax;
+  }
+
+  /**
+   * Called after #{@link DocValuesStats#accumulate(int)} was processed and verified that the document has a value for
+   * the field. Implementations should update the statistics based on the value of the current document.
+   *
+   * @param count
+   *          the updated number of documents with value for this field.
+   */
+  protected abstract void doAccumulate(int count) throws IOException;
+
+  /**
+   * Initializes this object with the given reader context. Returns whether stats can be computed for this segment (i.e.
+   * it does have the requested DocValues field).
+   */
+  protected abstract boolean init(LeafReaderContext contxt) throws IOException;
+
+  /** Returns whether the given document has a value for the requested DocValues field. */
+  protected abstract boolean hasValue(int doc) throws IOException;
+
+  final void accumulate(int doc) throws IOException {
+    if (hasValue(doc)) {
+      ++count;
+      doAccumulate(count);
+    } else {
+      ++missing;
+    }
+  }
+
+  final void addMissing() {
+    ++missing;
+  }
+
+  /** The field for which these stats were computed. */
+  public final String field() {
+    return field;
+  }
+
+  /** The number of documents which have a value of the field. */
+  public final int count() {
+    return count;
+  }
+
+  /** The number of documents which do not have a value of the field. */
+  public final int missing() {
+    return missing;
+  }
+
+  /** The minimum value of the field. Undefined when {@link #count} is zero. */
+  public final T min() {
+    return min;
+  }
+
+  /** The maximum value of the field. Undefined when {@link #count} is zero. */
+  public final T max() {
+    return max;
+  }
+
+  /** Holds statistics for a numeric DocValues field. */
+  public static abstract class NumericDocValuesStats<T extends Number> extends DocValuesStats<T> {
+
+    protected double mean = 0.0;
+
+    protected NumericDocValues ndv;
+
+    protected NumericDocValuesStats(String field, T initialMin, T initialMax) {
+      super(field, initialMin, initialMax);
+    }
+
+    @Override
+    protected final boolean init(LeafReaderContext contxt) throws IOException {
+      ndv = contxt.reader().getNumericDocValues(field);
+      return ndv != null;
+    }
+
+    @Override
+    protected boolean hasValue(int doc) throws IOException {
+      return ndv.advanceExact(doc);
+    }
+
+    /** The mean of all values of the field. Undefined when {@link #count} is zero. */
+    public final double mean() {
+      return mean;
+    }
+  }
+
+  /** Holds DocValues statistics for a numeric field storing {@code long} values. */
+  public static final class LongDocValuesStats extends NumericDocValuesStats<Long> {
+
+    public LongDocValuesStats(String description) {
+      super(description, Long.MAX_VALUE, Long.MIN_VALUE);
+    }
+
+    @Override
+    protected void doAccumulate(int count) throws IOException {
+      long val = ndv.longValue();
+      if (val > max) {
+        max = val;
+      }
+      if (val < min) {
+        min = val;
+      }
+      mean += (val - mean) / count;
+    }
+  }
+
+  /** Holds DocValues statistics for a numeric field storing {@code double} values. */
+  public static final class DoubleDocValuesStats extends NumericDocValuesStats<Double> {
+
+    public DoubleDocValuesStats(String description) {
+      super(description, Double.MAX_VALUE, Double.MIN_VALUE);
+    }
+
+    @Override
+    protected void doAccumulate(int count) throws IOException {
+      double val = Double.longBitsToDouble(ndv.longValue());
+      if (Double.compare(val, max) > 0) {
+        max = val;
+      }
+      if (Double.compare(val, min) < 0) {
+        min = val;
+      }
+      mean += (val - mean) / count;
+    }
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ad7152ad/lucene/misc/src/java/org/apache/lucene/search/DocValuesStatsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/search/DocValuesStatsCollector.java b/lucene/misc/src/java/org/apache/lucene/search/DocValuesStatsCollector.java
new file mode 100644
index 0000000..2b1fa4f
--- /dev/null
+++ b/lucene/misc/src/java/org/apache/lucene/search/DocValuesStatsCollector.java
@@ -0,0 +1,64 @@
+/*
+ * 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.lucene.search;
+
+import java.io.IOException;
+
+import org.apache.lucene.index.LeafReaderContext;
+
+/** A {@link Collector} which computes statistics for a DocValues field. */
+public class DocValuesStatsCollector implements Collector {
+
+  private final DocValuesStats<?> stats;
+
+  /** Creates a collector to compute statistics for a DocValues field using the given {@code stats}. */
+  public DocValuesStatsCollector(DocValuesStats<?> stats) {
+    this.stats = stats;
+  }
+
+  @Override
+  public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
+    boolean shouldProcess = stats.init(context);
+    if (!shouldProcess) {
+      // Stats cannot be computed for this segment, therefore consider all matching documents as a 'miss'. 
+      return new LeafCollector() {
+        @Override public void setScorer(Scorer scorer) throws IOException {}
+
+        @Override
+        public void collect(int doc) throws IOException {
+          // All matching documents in this reader are missing a value
+          stats.addMissing();
+        }
+      };
+    }
+
+    return new LeafCollector() {
+      @Override public void setScorer(Scorer scorer) throws IOException {}
+
+      @Override
+      public void collect(int doc) throws IOException {
+        stats.accumulate(doc);
+      }
+    };
+  }
+
+  @Override
+  public boolean needsScores() {
+    return false;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ad7152ad/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java b/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
new file mode 100644
index 0000000..65f82e6
--- /dev/null
+++ b/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
@@ -0,0 +1,166 @@
+/*
+ * 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.lucene.search;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.stream.DoubleStream;
+import java.util.stream.LongStream;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.DoubleDocValuesField;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.DocValuesStats.DoubleDocValuesStats;
+import org.apache.lucene.search.DocValuesStats.LongDocValuesStats;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+/** Unit tests for {@link DocValuesStatsCollector}. */
+public class TestDocValuesStatsCollector extends LuceneTestCase {
+
+  public void testNoDocsWithField() throws IOException {
+    try (Directory dir = newDirectory();
+        IndexWriter indexWriter = new IndexWriter(dir, newIndexWriterConfig())) {
+      int numDocs = TestUtil.nextInt(random(), 1, 100);
+      for (int i = 0; i < numDocs; i++) {
+        indexWriter.addDocument(new Document());
+      }
+
+      try (DirectoryReader reader = DirectoryReader.open(indexWriter)) {
+        IndexSearcher searcher = new IndexSearcher(reader);
+        LongDocValuesStats stats = new LongDocValuesStats("foo");
+        searcher.search(new MatchAllDocsQuery(), new DocValuesStatsCollector(stats));
+
+        assertEquals(0, stats.count());
+        assertEquals(numDocs, stats.missing());
+      }
+    }
+  }
+
+  public void testRandomDocsWithLongValues() throws IOException {
+    try (Directory dir = newDirectory();
+        IndexWriter indexWriter = new IndexWriter(dir, newIndexWriterConfig())) {
+      String field = "numeric";
+      int numDocs = TestUtil.nextInt(random(), 1, 100);
+      long[] docValues = new long[numDocs];
+      int nextVal = 1;
+      for (int i = 0; i < numDocs; i++) {
+        Document doc = new Document();
+        if (random().nextBoolean()) { // not all documents have a value
+          doc.add(new NumericDocValuesField(field, nextVal));
+          doc.add(new StringField("id", "doc" + i, Store.NO));
+          docValues[i] = nextVal;
+          ++nextVal;
+        }
+        indexWriter.addDocument(doc);
+      }
+
+      // 20% of cases delete some docs
+      if (random().nextDouble() < 0.2) {
+        for (int i = 0; i < numDocs; i++) {
+          if (random().nextBoolean()) {
+            indexWriter.deleteDocuments(new Term("id", "doc" + i));
+            docValues[i] = 0;
+          }
+        }
+      }
+
+      try (DirectoryReader reader = DirectoryReader.open(indexWriter)) {
+        IndexSearcher searcher = new IndexSearcher(reader);
+        LongDocValuesStats stats = new LongDocValuesStats(field);
+        searcher.search(new MatchAllDocsQuery(), new DocValuesStatsCollector(stats));
+
+        int expCount = (int) Arrays.stream(docValues).filter(v -> v > 0).count();
+        assertEquals(expCount, stats.count());
+        assertEquals(getZeroValues(docValues).count() - reader.numDeletedDocs(), stats.missing());
+        if (stats.count() > 0) {
+          assertEquals(getPositiveValues(docValues).max().getAsLong(), stats.max().longValue());
+          assertEquals(getPositiveValues(docValues).min().getAsLong(), stats.min().longValue());
+          assertEquals(getPositiveValues(docValues).average().getAsDouble(), stats.mean(), 0.00001);
+        }
+      }
+    }
+  }
+
+  public void testRandomDocsWithDoubleValues() throws IOException {
+    try (Directory dir = newDirectory();
+        IndexWriter indexWriter = new IndexWriter(dir, newIndexWriterConfig())) {
+      String field = "numeric";
+      int numDocs = TestUtil.nextInt(random(), 1, 100);
+      double[] docValues = new double[numDocs];
+      double nextVal = 1.0;
+      for (int i = 0; i < numDocs; i++) {
+        Document doc = new Document();
+        if (random().nextBoolean()) { // not all documents have a value
+          doc.add(new DoubleDocValuesField(field, nextVal));
+          doc.add(new StringField("id", "doc" + i, Store.NO));
+          docValues[i] = nextVal;
+          ++nextVal;
+        }
+        indexWriter.addDocument(doc);
+      }
+
+      // 20% of cases delete some docs
+      if (random().nextDouble() < 0.2) {
+        for (int i = 0; i < numDocs; i++) {
+          if (random().nextBoolean()) {
+            indexWriter.deleteDocuments(new Term("id", "doc" + i));
+            docValues[i] = 0;
+          }
+        }
+      }
+
+      try (DirectoryReader reader = DirectoryReader.open(indexWriter)) {
+        IndexSearcher searcher = new IndexSearcher(reader);
+        DoubleDocValuesStats stats = new DoubleDocValuesStats(field);
+        searcher.search(new MatchAllDocsQuery(), new DocValuesStatsCollector(stats));
+
+        int expCount = (int) Arrays.stream(docValues).filter(v -> v > 0).count();
+        assertEquals(expCount, stats.count());
+        assertEquals(getZeroValues(docValues).count() - reader.numDeletedDocs(), stats.missing());
+        if (stats.count() > 0) {
+          assertEquals(getPositiveValues(docValues).max().getAsDouble(), stats.max().doubleValue(), 0.00001);
+          assertEquals(getPositiveValues(docValues).min().getAsDouble(), stats.min().doubleValue(), 0.00001);
+          assertEquals(getPositiveValues(docValues).average().getAsDouble(), stats.mean(), 0.00001);
+        }
+      }
+    }
+  }
+
+  private static LongStream getPositiveValues(long[] docValues) {
+    return Arrays.stream(docValues).filter(v -> v > 0);
+  }
+
+  private static DoubleStream getPositiveValues(double[] docValues) {
+    return Arrays.stream(docValues).filter(v -> v > 0);
+  }
+
+  private static LongStream getZeroValues(long[] docValues) {
+    return Arrays.stream(docValues).filter(v -> v == 0);
+  }
+
+  private static DoubleStream getZeroValues(double[] docValues) {
+    return Arrays.stream(docValues).filter(v -> v == 0);
+  }
+
+}


[17/44] lucene-solr:jira/solr-9854: SOLR-4735: Use method in java.util.Objects instead of the forbidden methods in Guava's Preconditions class

Posted by ab...@apache.org.
SOLR-4735: Use method in java.util.Objects instead of the forbidden methods in Guava's Preconditions class


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

Branch: refs/heads/jira/solr-9854
Commit: 5f0637cc8569768ac9ce2a38cef5405163a974c0
Parents: 54e3510
Author: Shalin Shekhar Mangar <sh...@apache.org>
Authored: Fri Dec 16 20:53:20 2016 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Fri Dec 16 20:53:20 2016 +0530

----------------------------------------------------------------------
 .../core/src/java/org/apache/solr/metrics/SolrMetricManager.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5f0637cc/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java b/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
index abb79fb..2063689 100644
--- a/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
@@ -22,6 +22,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
+import java.util.Objects;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.Lock;
@@ -36,7 +37,6 @@ import com.codahale.metrics.MetricRegistry;
 import com.codahale.metrics.MetricSet;
 import com.codahale.metrics.SharedMetricRegistries;
 import com.codahale.metrics.Timer;
-import com.google.common.base.Preconditions;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.SolrInfoMBean;
 import org.apache.solr.core.SolrResourceLoader;
@@ -88,7 +88,7 @@ public class SolrMetricManager {
      *               name will match.
      */
     public PrefixFilter(String prefix) {
-      Preconditions.checkNotNull(prefix);
+      Objects.requireNonNull(prefix);
       this.prefix = prefix;
     }
 


[12/44] lucene-solr:jira/solr-9854: remove bad assertion

Posted by ab...@apache.org.
remove bad assertion


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

Branch: refs/heads/jira/solr-9854
Commit: 268d4ace3695ad3738402d623400fa4775b113ef
Parents: 295cab7
Author: Mike McCandless <mi...@apache.org>
Authored: Thu Dec 15 09:23:48 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Thu Dec 15 09:30:25 2016 -0500

----------------------------------------------------------------------
 .../src/java/org/apache/lucene/search/QueryUtils.java            | 4 ----
 1 file changed, 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/268d4ace/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java b/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java
index a3eaa80..ae4c890 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java
@@ -93,10 +93,6 @@ public class QueryUtils {
   public static void checkUnequal(Query q1, Query q2) {
     assertFalse(q1 + " equal to " + q2, q1.equals(q2));
     assertFalse(q2 + " equal to " + q1, q2.equals(q1));
-
-    // possible this test can fail on a hash collision... if that
-    // happens, please change test to use a different example.
-    assertTrue(q1.hashCode() != q2.hashCode());
   }
 
   /** deep check that explanations of a query 'score' correctly */


[05/44] lucene-solr:jira/solr-9854: Rename constructor parameter name

Posted by ab...@apache.org.
Rename constructor parameter name


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

Branch: refs/heads/jira/solr-9854
Commit: 22d9af41a435feaa3307880b7c7ed4f5860faa21
Parents: 85582da
Author: Shai Erera <sh...@apache.org>
Authored: Wed Dec 14 13:49:42 2016 +0200
Committer: Shai Erera <sh...@apache.org>
Committed: Wed Dec 14 13:49:42 2016 +0200

----------------------------------------------------------------------
 .../src/java/org/apache/lucene/search/DocValuesStats.java    | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22d9af41/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java b/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
index 38158cf..998bef4 100644
--- a/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
+++ b/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
@@ -125,8 +125,8 @@ public abstract class DocValuesStats<T> {
   /** Holds DocValues statistics for a numeric field storing {@code long} values. */
   public static final class LongDocValuesStats extends NumericDocValuesStats<Long> {
 
-    public LongDocValuesStats(String description) {
-      super(description, Long.MAX_VALUE, Long.MIN_VALUE);
+    public LongDocValuesStats(String field) {
+      super(field, Long.MAX_VALUE, Long.MIN_VALUE);
     }
 
     @Override
@@ -145,8 +145,8 @@ public abstract class DocValuesStats<T> {
   /** Holds DocValues statistics for a numeric field storing {@code double} values. */
   public static final class DoubleDocValuesStats extends NumericDocValuesStats<Double> {
 
-    public DoubleDocValuesStats(String description) {
-      super(description, Double.MAX_VALUE, Double.MIN_VALUE);
+    public DoubleDocValuesStats(String field) {
+      super(field, Double.MAX_VALUE, Double.MIN_VALUE);
     }
 
     @Override


[38/44] lucene-solr:jira/solr-9854: SOLR-9874: CREATEALIAS should fail if target collections don't exist

Posted by ab...@apache.org.
SOLR-9874: CREATEALIAS should fail if target collections don't exist


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

Branch: refs/heads/jira/solr-9854
Commit: 84bbb8f797db55f6cb203d87d150f2b3a2fe92cf
Parents: 4d81eee
Author: Tomas Fernandez Lobbe <tf...@apache.org>
Authored: Mon Dec 19 16:54:13 2016 -0800
Committer: Tomas Fernandez Lobbe <tf...@apache.org>
Committed: Mon Dec 19 16:54:13 2016 -0800

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  2 +
 .../solr/SolrMorphlineZkAliasTest.java          |  7 ---
 .../org/apache/solr/cloud/CreateAliasCmd.java   | 21 ++++++--
 .../solr/handler/admin/CollectionsHandler.java  |  2 +-
 .../apache/solr/cloud/AliasIntegrationTest.java | 50 ++++++++++++++++----
 .../solr/client/solrj/io/sql/JdbcTest.java      |  7 +--
 .../client/solrj/io/stream/JDBCStreamTest.java  |  9 ++--
 .../solrj/io/stream/StreamExpressionTest.java   |  7 +--
 .../client/solrj/io/stream/StreamingTest.java   | 10 ++--
 9 files changed, 81 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84bbb8f7/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 35a2bb7..d04d491 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -304,6 +304,8 @@ Other Changes
 
 * SOLR-8959: Refactored TestSegmentSorting out of TestMiniSolrCloudCluster (hossman)
 
+* SOLR-9874: Solr will reject CREATEALIAS requests if target collections don't exist (Tom�s Fern�ndez L�bbe)
+
 
 ==================  6.3.0 ==================
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84bbb8f7/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/SolrMorphlineZkAliasTest.java
----------------------------------------------------------------------
diff --git a/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/SolrMorphlineZkAliasTest.java b/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/SolrMorphlineZkAliasTest.java
index ddaf2f6..74c8824 100644
--- a/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/SolrMorphlineZkAliasTest.java
+++ b/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/SolrMorphlineZkAliasTest.java
@@ -91,13 +91,6 @@ public class SolrMorphlineZkAliasTest extends AbstractSolrMorphlineZkTestBase {
     Notifications.notifyRollbackTransaction(morphline);
     Notifications.notifyShutdown(morphline);
 
-    CollectionAdminRequest.createAlias("aliascollection", "collection1,collection2")
-        .processAndWait(cluster.getSolrClient(), TIMEOUT);
-
-    expectThrows(IllegalArgumentException.class, () -> {
-      parse("test-morphlines" + File.separator + "loadSolrBasic", "aliascollection");
-    });
-
   }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84bbb8f7/solr/core/src/java/org/apache/solr/cloud/CreateAliasCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/CreateAliasCmd.java b/solr/core/src/java/org/apache/solr/cloud/CreateAliasCmd.java
index b966ebd..339d75a 100644
--- a/solr/core/src/java/org/apache/solr/cloud/CreateAliasCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/CreateAliasCmd.java
@@ -17,8 +17,11 @@
  */
 package org.apache.solr.cloud;
 
+import static org.apache.solr.common.params.CommonParams.NAME;
+
 import java.lang.invoke.MethodHandles;
 import java.util.HashMap;
+import java.util.Locale;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
@@ -35,8 +38,6 @@ import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.solr.common.params.CommonParams.NAME;
-
 
 public class CreateAliasCmd implements Cmd {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@@ -52,10 +53,12 @@ public class CreateAliasCmd implements Cmd {
     String aliasName = message.getStr(NAME);
     String collections = message.getStr("collections");
 
-    Map<String, Map<String, String>> newAliasesMap = new HashMap<>();
-    Map<String, String> newCollectionAliasesMap = new HashMap<>();
     ZkStateReader zkStateReader = ocmh.zkStateReader;
     Map<String, String> prevColAliases = zkStateReader.getAliases().getCollectionAliasMap();
+    validateAllCollectionsExist(collections, prevColAliases, zkStateReader.getClusterState());
+
+    Map<String, Map<String, String>> newAliasesMap = new HashMap<>();
+    Map<String, String> newCollectionAliasesMap = new HashMap<>();
     if (prevColAliases != null) {
       newCollectionAliasesMap.putAll(prevColAliases);
     }
@@ -81,6 +84,16 @@ public class CreateAliasCmd implements Cmd {
     }
   }
 
+  private void validateAllCollectionsExist(String collections, Map<String,String> prevColAliases, ClusterState clusterState) {
+    String[] collectionArr = collections.split(",");
+    for (String collection:collectionArr) {
+      if (clusterState.getCollectionOrNull(collection) == null && (prevColAliases == null || !prevColAliases.containsKey(collection))) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+            String.format(Locale.ROOT,  "Can't create collection alias for collections='%s', '%s' is not an existing collection or alias", collections, collection));
+      }
+    }
+  }
+
   private void checkForAlias(String name, String value) {
 
     TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84bbb8f7/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
index 1915176..e683e96 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
@@ -437,7 +437,7 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
       return null;
     }),
     CREATEALIAS_OP(CREATEALIAS, (req, rsp, h) -> {
-      final String aliasName = SolrIdentifierValidator.validateAliasName(req.getParams().get(NAME));
+      SolrIdentifierValidator.validateAliasName(req.getParams().get(NAME));
       return req.getParams().required().getAll(null, NAME, "collections");
     }),
     DELETEALIAS_OP(DELETEALIAS, (req, rsp, h) -> req.getParams().required().getAll(null, NAME)),

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84bbb8f7/solr/core/src/test/org/apache/solr/cloud/AliasIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/AliasIntegrationTest.java b/solr/core/src/test/org/apache/solr/cloud/AliasIntegrationTest.java
index 50e84e4..6ca072b 100644
--- a/solr/core/src/test/org/apache/solr/cloud/AliasIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/AliasIntegrationTest.java
@@ -16,8 +16,6 @@
  */
 package org.apache.solr.cloud;
 
-import java.lang.invoke.MethodHandles;
-
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
@@ -28,13 +26,9 @@ import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.common.SolrException;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 public class AliasIntegrationTest extends SolrCloudTestCase {
 
-  private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  
   @BeforeClass
   public static void setupCluster() throws Exception {
     configureCluster(2)
@@ -160,9 +154,49 @@ public class AliasIntegrationTest extends SolrCloudTestCase {
       cluster.getSolrClient().query(q);
     });
     assertTrue("Unexpected exception message: " + e.getMessage(), e.getMessage().contains("Collection not found: testalias"));
-
-    logger.info("### FINISHED ACTUAL TEST");
   }
 
+  public void testErrorChecks() throws Exception {
+
+    CollectionAdminRequest.createCollection("testErrorChecks-collection", "conf", 2, 1).process(cluster.getSolrClient());
+    waitForState("Expected testErrorChecks-collection to be created with 2 shards and 1 replica", "testErrorChecks-collection", clusterShape(2, 1));
+    
+    ignoreException(".");
+    
+    // Invalid Alias name
+    SolrException e = expectThrows(SolrException.class, () -> {
+      CollectionAdminRequest.createAlias("test:alias", "testErrorChecks-collection").process(cluster.getSolrClient());
+    });
+    assertEquals(SolrException.ErrorCode.BAD_REQUEST, SolrException.ErrorCode.getErrorCode(e.code()));
+
+    // Target collection doesn't exists
+    e = expectThrows(SolrException.class, () -> {
+      CollectionAdminRequest.createAlias("testalias", "doesnotexist").process(cluster.getSolrClient());
+    });
+    assertEquals(SolrException.ErrorCode.BAD_REQUEST, SolrException.ErrorCode.getErrorCode(e.code()));
+    assertTrue(e.getMessage().contains("Can't create collection alias for collections='doesnotexist', 'doesnotexist' is not an existing collection or alias"));
+
+    // One of the target collections doesn't exist
+    e = expectThrows(SolrException.class, () -> {
+      CollectionAdminRequest.createAlias("testalias", "testErrorChecks-collection,doesnotexist").process(cluster.getSolrClient());
+    });
+    assertEquals(SolrException.ErrorCode.BAD_REQUEST, SolrException.ErrorCode.getErrorCode(e.code()));
+    assertTrue(e.getMessage().contains("Can't create collection alias for collections='testErrorChecks-collection,doesnotexist', 'doesnotexist' is not an existing collection or alias"));
+
+    // Valid
+    CollectionAdminRequest.createAlias("testalias", "testErrorChecks-collection").process(cluster.getSolrClient());
+    CollectionAdminRequest.createAlias("testalias2", "testalias").process(cluster.getSolrClient());
+
+    // Alias + invalid
+    e = expectThrows(SolrException.class, () -> {
+      CollectionAdminRequest.createAlias("testalias3", "testalias2,doesnotexist").process(cluster.getSolrClient());
+    });
+    assertEquals(SolrException.ErrorCode.BAD_REQUEST, SolrException.ErrorCode.getErrorCode(e.code()));
+    unIgnoreException(".");
+
+    CollectionAdminRequest.deleteAlias("testalias").process(cluster.getSolrClient());
+    CollectionAdminRequest.deleteAlias("testalias2").process(cluster.getSolrClient());
+    CollectionAdminRequest.deleteCollection("testErrorChecks-collection");
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84bbb8f7/solr/solrj/src/test/org/apache/solr/client/solrj/io/sql/JdbcTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/sql/JdbcTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/sql/JdbcTest.java
index d0a600d..885fe82 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/sql/JdbcTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/sql/JdbcTest.java
@@ -62,16 +62,17 @@ public class JdbcTest extends SolrCloudTestCase {
 
     String collection;
     boolean useAlias = random().nextBoolean();
-    if(useAlias) {
+    if (useAlias) {
       collection = COLLECTIONORALIAS + "_collection";
-      CollectionAdminRequest.createAlias(COLLECTIONORALIAS, collection).process(cluster.getSolrClient());
     } else {
       collection = COLLECTIONORALIAS;
     }
-
     CollectionAdminRequest.createCollection(collection, "conf", 2, 1).process(cluster.getSolrClient());
     AbstractDistribZkTestBase.waitForRecoveriesToFinish(collection, cluster.getSolrClient().getZkStateReader(),
         false, true, DEFAULT_TIMEOUT);
+    if (useAlias) {
+      CollectionAdminRequest.createAlias(COLLECTIONORALIAS, collection).process(cluster.getSolrClient());
+    }
 
     new UpdateRequest()
         .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1", "testnull_i", null)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84bbb8f7/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/JDBCStreamTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/JDBCStreamTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/JDBCStreamTest.java
index c661fa2..e55c837 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/JDBCStreamTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/JDBCStreamTest.java
@@ -62,18 +62,19 @@ public class JDBCStreamTest extends SolrCloudTestCase {
         .addConfig("conf", getFile("solrj").toPath().resolve("solr").resolve("configsets").resolve("streaming").resolve("conf"))
         .configure();
 
-    String collection;
     boolean useAlias = random().nextBoolean();
-    if(useAlias) {
+    String collection;
+    if (useAlias) {
       collection = COLLECTIONORALIAS + "_collection";
-      CollectionAdminRequest.createAlias(COLLECTIONORALIAS, collection).process(cluster.getSolrClient());
     } else {
       collection = COLLECTIONORALIAS;
     }
-
     CollectionAdminRequest.createCollection(collection, "conf", 2, 1).process(cluster.getSolrClient());
     AbstractDistribZkTestBase.waitForRecoveriesToFinish(collection, cluster.getSolrClient().getZkStateReader(),
         false, true, TIMEOUT);
+    if (useAlias) {
+      CollectionAdminRequest.createAlias(COLLECTIONORALIAS, collection).process(cluster.getSolrClient());
+    }
   }
 
   @BeforeClass

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84bbb8f7/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
index ff5a062..7d48c0e 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
@@ -80,16 +80,17 @@ public class StreamExpressionTest extends SolrCloudTestCase {
 
     String collection;
     useAlias = random().nextBoolean();
-    if(useAlias) {
+    if (useAlias) {
       collection = COLLECTIONORALIAS + "_collection";
-      CollectionAdminRequest.createAlias(COLLECTIONORALIAS, collection).process(cluster.getSolrClient());
     } else {
       collection = COLLECTIONORALIAS;
     }
-
     CollectionAdminRequest.createCollection(collection, "conf", 2, 1).process(cluster.getSolrClient());
     AbstractDistribZkTestBase.waitForRecoveriesToFinish(collection, cluster.getSolrClient().getZkStateReader(),
         false, true, TIMEOUT);
+    if (useAlias) {
+      CollectionAdminRequest.createAlias(COLLECTIONORALIAS, collection).process(cluster.getSolrClient());
+    }
   }
 
   @Before

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84bbb8f7/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamingTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamingTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamingTest.java
index d430409..cf60bc3 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamingTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamingTest.java
@@ -94,15 +94,17 @@ public static void configureCluster() throws Exception {
 
   String collection;
   useAlias = random().nextBoolean();
-  if(useAlias) {
+  if (useAlias) {
     collection = COLLECTIONORALIAS + "_collection";
-    CollectionAdminRequest.createAlias(COLLECTIONORALIAS, collection).process(cluster.getSolrClient());
   } else {
     collection = COLLECTIONORALIAS;
   }
-
   CollectionAdminRequest.createCollection(collection, "conf", numShards, 1).process(cluster.getSolrClient());
-  AbstractDistribZkTestBase.waitForRecoveriesToFinish(collection, cluster.getSolrClient().getZkStateReader(), false, true, DEFAULT_TIMEOUT);
+  AbstractDistribZkTestBase.waitForRecoveriesToFinish(collection, cluster.getSolrClient().getZkStateReader(),
+      false, true, DEFAULT_TIMEOUT);
+  if (useAlias) {
+    CollectionAdminRequest.createAlias(COLLECTIONORALIAS, collection).process(cluster.getSolrClient());
+  }
 
   zkHost = cluster.getZkServer().getZkAddress();
   streamFactory.withCollectionZkHost(COLLECTIONORALIAS, zkHost);


[10/44] lucene-solr:jira/solr-9854: LUCENE-7592: if segments file is truncated, throw CorruptIndexException

Posted by ab...@apache.org.
LUCENE-7592: if segments file is truncated, throw CorruptIndexException


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

Branch: refs/heads/jira/solr-9854
Commit: e4f31fab2f98b7af6d2ec12a2eb3456521b446df
Parents: 6525bb5
Author: Mike McCandless <mi...@apache.org>
Authored: Wed Dec 14 18:00:51 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Wed Dec 14 18:00:51 2016 -0500

----------------------------------------------------------------------
 lucene/CHANGES.txt                                            | 4 ++++
 .../core/src/java/org/apache/lucene/index/SegmentInfos.java   | 7 ++++++-
 2 files changed, 10 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e4f31fab/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 15b89f0..f38c0d5 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -138,6 +138,10 @@ Improvements
   necessarily refer to that field (AKA requireFieldMatch==false). Disabled by default.
   See UH get/setFieldMatcher. (Jim Ferenczi via David Smiley)
 
+* LUCENE-7592: If the segments file is truncated, we now throw
+  CorruptIndexException instead of the more confusing EOFException
+  (Mike Drob via Mike McCandless)
+
 Optimizations
 
 * LUCENE-7568: Optimize merging when index sorting is used but the

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e4f31fab/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
index 8f627cd..3e8b1f8 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
@@ -17,6 +17,7 @@
 package org.apache.lucene.index;
 
 
+import java.io.EOFException;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.util.ArrayList;
@@ -277,7 +278,11 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
     long generation = generationFromSegmentsFileName(segmentFileName);
     //System.out.println(Thread.currentThread() + ": SegmentInfos.readCommit " + segmentFileName);
     try (ChecksumIndexInput input = directory.openChecksumInput(segmentFileName, IOContext.READ)) {
-      return readCommit(directory, input, generation);
+      try {
+        return readCommit(directory, input, generation);
+      } catch (EOFException e) {
+        throw new CorruptIndexException("Unexpected end of file while reading index.", input, e);
+      }
     }
   }
 


[27/44] lucene-solr:jira/solr-9854: SOLR-4735 Update CHANGES.txt

Posted by ab...@apache.org.
SOLR-4735 Update CHANGES.txt


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

Branch: refs/heads/jira/solr-9854
Commit: f56da1df6e92da5f1ab524caf62d30ea3a3ceede
Parents: 92ef10d
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Mon Dec 19 13:45:05 2016 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Mon Dec 19 13:45:05 2016 +0100

----------------------------------------------------------------------
 solr/CHANGES.txt | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f56da1df/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index a0f11f1..0ab961c 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -167,6 +167,10 @@ New Features
 * SOLR-9844: FieldCache information fetched via the mbeans handler or seen via the UI now displays the total size used.
   The individual cache entries in the response are now formatted better as well. (Varun Thacker)
 
+* SOLR-4735: Improve metrics reporting. This uses the dropwizard metric library, adding an internal API
+  for registering and reporting metrics from Solr components. Several new metrics and an improved JMX
+  reporter have been added (Alan Woodward, Jeff Wartes, Christine Poerschke, Kelvin Wong, shalin, ab)
+
 * SOLR-9788: Use instrumented jetty classes provided by the dropwizard metric library. (shalin)
 
 * SOLR-9805: Use metrics-jvm library to instrument jvm internals such as GC, memory usage and others. (shalin)


[43/44] lucene-solr:jira/solr-9854: Merge branch 'feature/metrics' into jira/solr-9854

Posted by ab...@apache.org.
Merge branch 'feature/metrics' into jira/solr-9854


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

Branch: refs/heads/jira/solr-9854
Commit: 3562ef76293f5b419cdf4497822c615fe65efebf
Parents: ce8ab8c aed34a4
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Tue Dec 20 11:42:41 2016 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Tue Dec 20 11:42:41 2016 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  17 +
 .../lucene70/Lucene70DocValuesConsumer.java     | 163 +++++--
 .../lucene70/Lucene70DocValuesFormat.java       |   5 +-
 .../lucene70/Lucene70DocValuesProducer.java     | 220 +++++++--
 .../org/apache/lucene/index/LeafReader.java     |   2 +-
 .../apache/lucene/index/PrefixCodedTerms.java   |   4 +-
 .../org/apache/lucene/index/SegmentInfos.java   |   7 +-
 .../java/org/apache/lucene/util/LongValues.java |   9 +
 .../apache/lucene/util/packed/DirectWriter.java |   8 +-
 .../lucene70/TestLucene70DocValuesFormat.java   | 152 +++++++
 .../org/apache/lucene/facet/FacetQuery.java     |  52 +++
 .../apache/lucene/facet/MultiFacetQuery.java    |  61 +++
 .../org/apache/lucene/facet/TestFacetQuery.java | 103 +++++
 lucene/ivy-versions.properties                  |   2 -
 .../apache/lucene/search/DocValuesStats.java    | 409 +++++++++++++++++
 .../lucene/search/DocValuesStatsCollector.java  |  64 +++
 .../search/TestDocValuesStatsCollector.java     | 448 +++++++++++++++++++
 .../apache/lucene/document/RangeFieldQuery.java |   1 +
 .../lucene/search/DocValuesNumbersQuery.java    |  26 +-
 .../lucene/search/DocValuesTermsQuery.java      |  49 +-
 .../org/apache/lucene/search/LongHashSet.java   | 156 +++++++
 .../search/BaseRangeFieldQueryTestCase.java     |   2 +-
 .../apache/lucene/search/LongHashSetTests.java  | 100 +++++
 .../lucene/search/TestDocValuesTermsQuery.java  |   1 +
 .../org/apache/lucene/search/QueryUtils.java    |   4 -
 solr/CHANGES.txt                                |  26 +-
 .../plugin/AnalyticsStatisticsCollector.java    |   4 +-
 .../solr/collection1/conf/solrconfig.xml        |   5 -
 solr/contrib/ltr/example/solrconfig.xml         |  11 -
 .../solr/solrcloud/conf/solrconfig.xml          |  11 -
 .../OverseerAutoReplicaFailoverThread.java      |   9 +-
 .../apache/solr/cloud/OverseerStatusCmd.java    |   4 +-
 .../org/apache/solr/core/CoreContainer.java     |  43 +-
 .../java/org/apache/solr/core/SolrConfig.java   |   2 +-
 .../src/java/org/apache/solr/core/SolrCore.java | 127 +++---
 .../apache/solr/handler/RequestHandlerBase.java |  18 +-
 .../solr/handler/admin/MetricsHandler.java      | 164 +++++++
 .../solr/metrics/SolrCoreMetricManager.java     |  20 +-
 .../apache/solr/metrics/SolrMetricManager.java  | 120 +++--
 .../apache/solr/metrics/SolrMetricProducer.java |   3 +-
 .../apache/solr/metrics/SolrMetricReporter.java |   5 +-
 .../solr/metrics/reporters/SolrJmxReporter.java |   6 +-
 .../apache/solr/search/SolrFieldCacheMBean.java |   6 +-
 .../solr/security/PermissionNameProvider.java   |   1 +
 .../apache/solr/servlet/SolrDispatchFilter.java |  33 +-
 .../SolrMetricsServletContextListener.java      |  29 --
 .../apache/solr/servlet/SolrRequestParsers.java |  31 +-
 .../solr/uninverting/UninvertingReader.java     |  25 +-
 .../solr/util/SolrFileCleaningTracker.java      | 147 ++++++
 .../org/apache/solr/util/stats/MetricUtils.java | 144 ++++++
 .../org/apache/solr/util/stats/TimerUtils.java  |  58 ---
 .../solr/collection1/conf/solrconfig_perf.xml   |   1 -
 .../cloud/SegmentTerminateEarlyTestState.java   |  12 +-
 .../solr/cloud/TestMiniSolrCloudCluster.java    |  50 ---
 .../apache/solr/cloud/TestSegmentSorting.java   | 133 ++++++
 .../solr/handler/admin/MetricsHandlerTest.java  |  97 ++++
 .../solr/metrics/SolrCoreMetricManagerTest.java |  42 +-
 .../solr/metrics/SolrMetricManagerTest.java     |  83 ++--
 .../solr/metrics/SolrMetricReporterTest.java    |   4 +-
 .../solr/metrics/SolrMetricTestUtils.java       |   6 +-
 .../metrics/SolrMetricsIntegrationTest.java     |  16 +-
 .../metrics/reporters/MockMetricReporter.java   |   6 +-
 .../metrics/reporters/SolrJmxReporterTest.java  |  26 +-
 .../org/apache/solr/search/TestRealTimeGet.java |   2 +-
 .../solr/search/TestSolrFieldCacheMBean.java    |   3 +-
 .../apache/solr/search/TestStressLucene.java    |   2 +-
 .../apache/solr/search/TestStressRecovery.java  |   2 +-
 .../apache/solr/search/TestStressReorder.java   |   5 +-
 .../solr/search/TestStressUserVersions.java     |   4 +-
 .../apache/solr/search/TestStressVersions.java  |   2 +-
 .../apache/solr/util/stats/MetricUtilsTest.java |  58 +++
 .../apache/solr/util/stats/TimerUtilsTest.java  |  58 ---
 .../example-DIH/solr/db/conf/solrconfig.xml     |  11 -
 .../example-DIH/solr/mail/conf/solrconfig.xml   |  11 -
 .../example-DIH/solr/rss/conf/solrconfig.xml    |  11 -
 .../example-DIH/solr/solr/conf/solrconfig.xml   |  11 -
 .../example-DIH/solr/tika/conf/solrconfig.xml   |  11 -
 solr/example/files/conf/solrconfig.xml          |  11 -
 solr/licenses/metrics-json-3.1.2.jar.sha1       |   1 -
 solr/licenses/metrics-jvm-3.1.2.jar.sha1        |   1 +
 solr/licenses/metrics-jvm-LICENSE-ASL.txt       | 203 +++++++++
 solr/licenses/metrics-jvm-NOTICE.txt            |  12 +
 solr/licenses/metrics-servlets-3.1.2.jar.sha1   |   1 -
 solr/server/ivy.xml                             |   5 -
 .../basic_configs/conf/solrconfig.xml           |  11 -
 .../conf/solrconfig.xml                         |  11 -
 .../conf/solrconfig.xml                         |  11 -
 .../apache/solr/common/params/CommonParams.java |   4 +-
 .../solr/client/solrj/LargeVolumeTestBase.java  |   3 +-
 .../solr/client/solrj/SolrExampleTestBase.java  |   1 -
 .../java/org/apache/solr/SolrJettyTestBase.java |   2 -
 .../java/org/apache/solr/SolrTestCaseJ4.java    |   2 +-
 .../java/org/apache/solr/util/TestHarness.java  |  11 +-
 solr/webapp/web/WEB-INF/web.xml                 |  17 -
 94 files changed, 3341 insertions(+), 750 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3562ef76/solr/core/src/java/org/apache/solr/core/SolrCore.java
----------------------------------------------------------------------


[16/44] lucene-solr:jira/solr-9854: SOLR-9805: Added sha, notice and license files for metrics-jvm library

Posted by ab...@apache.org.
SOLR-9805: Added sha, notice and license files for metrics-jvm library


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

Branch: refs/heads/jira/solr-9854
Commit: 54e35102fe0d18f8a14b3cbd1d368c5d47cfb706
Parents: 054f38f
Author: Shalin Shekhar Mangar <sh...@apache.org>
Authored: Fri Dec 16 20:50:07 2016 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Fri Dec 16 20:50:07 2016 +0530

----------------------------------------------------------------------
 solr/licenses/metrics-jvm-3.1.2.jar.sha1  |   1 +
 solr/licenses/metrics-jvm-LICENSE-ASL.txt | 203 +++++++++++++++++++++++++
 solr/licenses/metrics-jvm-NOTICE.txt      |  12 ++
 3 files changed, 216 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/54e35102/solr/licenses/metrics-jvm-3.1.2.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/metrics-jvm-3.1.2.jar.sha1 b/solr/licenses/metrics-jvm-3.1.2.jar.sha1
new file mode 100644
index 0000000..519fcdd
--- /dev/null
+++ b/solr/licenses/metrics-jvm-3.1.2.jar.sha1
@@ -0,0 +1 @@
+ed364e77218e50fdcdebce4d982cb4d1f4a8c187

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/54e35102/solr/licenses/metrics-jvm-LICENSE-ASL.txt
----------------------------------------------------------------------
diff --git a/solr/licenses/metrics-jvm-LICENSE-ASL.txt b/solr/licenses/metrics-jvm-LICENSE-ASL.txt
new file mode 100644
index 0000000..ccb320c
--- /dev/null
+++ b/solr/licenses/metrics-jvm-LICENSE-ASL.txt
@@ -0,0 +1,203 @@
+
+                                 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 2010-2012 Coda Hale and Yammer, Inc.
+
+   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.
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/54e35102/solr/licenses/metrics-jvm-NOTICE.txt
----------------------------------------------------------------------
diff --git a/solr/licenses/metrics-jvm-NOTICE.txt b/solr/licenses/metrics-jvm-NOTICE.txt
new file mode 100644
index 0000000..b4c6298
--- /dev/null
+++ b/solr/licenses/metrics-jvm-NOTICE.txt
@@ -0,0 +1,12 @@
+Metrics
+Copyright 2010-2013 Coda Hale and Yammer, Inc.
+
+This product includes software developed by Coda Hale and Yammer, Inc.
+
+This product includes code derived from the JSR-166 project (ThreadLocalRandom, Striped64,
+LongAdder), which was released with the following comments:
+
+    Written by Doug Lea with assistance from members of JCP JSR-166
+    Expert Group and released to the public domain, as explained at
+    http://creativecommons.org/publicdomain/zero/1.0/
+


[31/44] lucene-solr:jira/solr-9854: SOLR-9860: Enable configuring invariantParams via HttpSolrClient.Builder

Posted by ab...@apache.org.
SOLR-9860: Enable configuring invariantParams via HttpSolrClient.Builder


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

Branch: refs/heads/jira/solr-9854
Commit: 2c4c5040eecdf7ab63d81e6e5a2e519891684d59
Parents: a1a8b28
Author: Ishan Chattopadhyaya <is...@apache.org>
Authored: Mon Dec 19 21:01:20 2016 +0530
Committer: Ishan Chattopadhyaya <is...@apache.org>
Committed: Mon Dec 19 21:01:20 2016 +0530

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  3 ++
 .../impl/DelegationTokenHttpSolrClient.java     | 34 +++++++++++-
 .../solr/client/solrj/impl/HttpSolrClient.java  | 56 ++++++++++++++++----
 .../solrj/impl/BasicHttpSolrClientTest.java     | 31 +++++++++++
 4 files changed, 111 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2c4c5040/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index f783934..35a2bb7 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -176,6 +176,9 @@ New Features
 * SOLR-9513: Generic authentication plugins (GenericHadoopAuthPlugin and ConfigurableInternodeAuthHadoopPlugin) that delegate
   all functionality to Hadoop authentication framework. (Hrishikesh Gadre via Ishan Chattopadhyaya)
 
+* SOLR-9860: Enable configuring invariantParams via HttpSolrClient.Builder (Hrishikesh Gadre, Ishan Chattopadhyaya)
+ 
+
 Optimizations
 ----------------------
 * SOLR-9704: Facet Module / JSON Facet API: Optimize blockChildren facets that have

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2c4c5040/solr/solrj/src/java/org/apache/solr/client/solrj/impl/DelegationTokenHttpSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/DelegationTokenHttpSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/DelegationTokenHttpSolrClient.java
index c14e9ed..ab8175d 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/DelegationTokenHttpSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/DelegationTokenHttpSolrClient.java
@@ -33,8 +33,18 @@ import org.apache.solr.common.params.SolrParams;
 
 public class DelegationTokenHttpSolrClient extends HttpSolrClient {
   public final static String DELEGATION_TOKEN_PARAM = "delegation";
-  private final String delegationToken;
 
+  /**
+   * This constructor is deprecated in favor of passing delegation token via
+   * {@linkplain org.apache.solr.client.solrj.impl.HttpSolrClient.Builder#withInvariantParams(ModifiableSolrParams)}.
+   *
+   * @param baseURL The base url to communicate with the Solr server
+   * @param client Http client instance to use for communication
+   * @param parser Response parser instance to use to decode response from Solr server
+   * @param allowCompression Should compression be allowed ?
+   * @param delegationToken The delegation token string.
+   */
+  @Deprecated
   public DelegationTokenHttpSolrClient(String baseURL,
                                        HttpClient client,
                                        ResponseParser parser,
@@ -44,12 +54,32 @@ public class DelegationTokenHttpSolrClient extends HttpSolrClient {
     if (delegationToken == null) {
       throw new IllegalArgumentException("Delegation token cannot be null");
     }
-    this.delegationToken = delegationToken;
     setQueryParams(new TreeSet<>(Arrays.asList(DELEGATION_TOKEN_PARAM)));
     invariantParams = new ModifiableSolrParams();
     invariantParams.set(DELEGATION_TOKEN_PARAM, delegationToken);
   }
 
+  /**
+   * This constructor is defined at "protected" scope. Ideally applications should
+   * use {@linkplain org.apache.solr.client.solrj.impl.HttpSolrClient.Builder} instance
+   * to configure this Solr client instance.
+   *
+   * @param baseURL The base url to communicate with the Solr server
+   * @param client Http client instance to use for communication
+   * @param parser Response parser instance to use to decode response from Solr server
+   * @param allowCompression Should compression be allowed ?
+   * @param invariantParams The parameters which should be passed with every request.
+   */
+  protected DelegationTokenHttpSolrClient(String baseURL,
+      HttpClient client,
+      ResponseParser parser,
+      boolean allowCompression,
+      ModifiableSolrParams invariantParams) {
+    super(baseURL, client, parser, allowCompression, invariantParams);
+
+    setQueryParams(new TreeSet<>(Arrays.asList(DELEGATION_TOKEN_PARAM)));
+  }
+
   @Override
   protected HttpRequestBase createMethod(final SolrRequest request, String collection) throws IOException, SolrServerException {
     SolrParams params = request.getParams();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2c4c5040/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java
index 3e2355c..5b272fc 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java
@@ -29,6 +29,7 @@ import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Locale;
+import java.util.Objects;
 import java.util.Set;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
@@ -212,7 +213,23 @@ public class HttpSolrClient extends SolrClient {
     
     this.parser = parser;
   }
-  
+
+  /**
+   * The consturctor.
+   *
+   * @param baseURL The base url to communicate with the Solr server
+   * @param client Http client instance to use for communication
+   * @param parser Response parser instance to use to decode response from Solr server
+   * @param allowCompression Should compression be allowed ?
+   * @param invariantParams The parameters which should be included with every request.
+   */
+  protected HttpSolrClient(String baseURL, HttpClient client, ResponseParser parser, boolean allowCompression,
+      ModifiableSolrParams invariantParams) {
+    this(baseURL, client, parser, allowCompression);
+
+    this.invariantParams = invariantParams;
+  }
+
   public Set<String> getQueryParams() {
     return queryParams;
   }
@@ -756,7 +773,7 @@ public class HttpSolrClient extends SolrClient {
     private HttpClient httpClient;
     private ResponseParser responseParser;
     private boolean compression;
-    private String delegationToken;
+    private ModifiableSolrParams invariantParams = new ModifiableSolrParams();
 
     public Builder() {
       this.responseParser = new BinaryResponseParser();
@@ -778,7 +795,7 @@ public class HttpSolrClient extends SolrClient {
       this.baseSolrUrl = baseSolrUrl;
       this.responseParser = new BinaryResponseParser();
     }
-    
+
     /**
      * Provides a {@link HttpClient} for the builder to use when creating clients.
      */
@@ -786,7 +803,7 @@ public class HttpSolrClient extends SolrClient {
       this.httpClient = httpClient;
       return this;
     }
-    
+
     /**
      * Provides a {@link ResponseParser} for created clients to use when handling requests.
      */
@@ -794,7 +811,7 @@ public class HttpSolrClient extends SolrClient {
       this.responseParser = responseParser;
       return this;
     }
-    
+
     /**
      * Chooses whether created {@link HttpSolrClient}s use compression by default.
      */
@@ -807,8 +824,7 @@ public class HttpSolrClient extends SolrClient {
      * Use a delegation token for authenticating via the KerberosPlugin
      */
     public Builder withKerberosDelegationToken(String delegationToken) {
-      this.delegationToken = delegationToken;
-      return this;
+      return withDelegationToken(delegationToken);
     }
 
     @Deprecated
@@ -816,9 +832,26 @@ public class HttpSolrClient extends SolrClient {
      * @deprecated use {@link withKerberosDelegationToken(String)} instead
      */
     public Builder withDelegationToken(String delegationToken) {
-      this.delegationToken = delegationToken;
+      if (this.invariantParams.get(DelegationTokenHttpSolrClient.DELEGATION_TOKEN_PARAM) != null) {
+        throw new IllegalStateException(DelegationTokenHttpSolrClient.DELEGATION_TOKEN_PARAM + " is already defined!");
+      }
+      this.invariantParams.add(DelegationTokenHttpSolrClient.DELEGATION_TOKEN_PARAM, delegationToken);
+      return this;
+    }
+
+    public Builder withInvariantParams(ModifiableSolrParams params) {
+      Objects.requireNonNull(params, "params must be non null!");
+
+      for (String name : params.getParameterNames()) {
+        if (this.invariantParams.get(name) != null) {
+          throw new IllegalStateException("parameter " + name + " is redefined.");
+        }
+      }
+
+      this.invariantParams.add(params);
       return this;
     }
+
     /**
      * Create a {@link HttpSolrClient} based on provided configuration.
      */
@@ -826,10 +859,11 @@ public class HttpSolrClient extends SolrClient {
       if (baseSolrUrl == null) {
         throw new IllegalArgumentException("Cannot create HttpSolrClient without a valid baseSolrUrl!");
       }
-      if (delegationToken == null) {
-        return new HttpSolrClient(baseSolrUrl, httpClient, responseParser, compression);
+
+      if (this.invariantParams.get(DelegationTokenHttpSolrClient.DELEGATION_TOKEN_PARAM) == null) {
+        return new HttpSolrClient(baseSolrUrl, httpClient, responseParser, compression, invariantParams);
       } else {
-        return new DelegationTokenHttpSolrClient(baseSolrUrl, httpClient, responseParser, compression, delegationToken);
+        return new DelegationTokenHttpSolrClient(baseSolrUrl, httpClient, responseParser, compression, invariantParams);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2c4c5040/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BasicHttpSolrClientTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BasicHttpSolrClientTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BasicHttpSolrClientTest.java
index 11d2784..06ae8b8 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BasicHttpSolrClientTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BasicHttpSolrClientTest.java
@@ -54,6 +54,7 @@ import org.apache.http.impl.client.CloseableHttpClient;
 import org.apache.http.impl.cookie.BasicClientCookie;
 import org.apache.http.protocol.HttpContext;
 import org.apache.solr.SolrJettyTestBase;
+import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrRequest.METHOD;
@@ -77,6 +78,7 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+
 public class BasicHttpSolrClientTest extends SolrJettyTestBase {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@@ -818,4 +820,33 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
       verifyServletState(client, req);
     }
   }
+
+  @Test
+  public void testInvariantParams() throws IOException {
+    try(HttpSolrClient createdClient = new HttpSolrClient.Builder()
+        .withBaseSolrUrl(jetty.getBaseUrl().toString())
+        .withInvariantParams(SolrTestCaseJ4.params("param", "value"))
+        .build()) {
+      assertEquals("value", createdClient.getInvariantParams().get("param"));
+    }
+
+    try(HttpSolrClient createdClient = new HttpSolrClient.Builder()
+        .withBaseSolrUrl(jetty.getBaseUrl().toString())
+        .withInvariantParams(SolrTestCaseJ4.params("fq", "fq1", "fq", "fq2"))
+        .build()) {
+      assertEquals(2, createdClient.getInvariantParams().getParams("fq").length);
+    }
+
+    try(HttpSolrClient createdClient = new HttpSolrClient.Builder()
+        .withBaseSolrUrl(jetty.getBaseUrl().toString())
+        .withDelegationToken("mydt")
+        .withInvariantParams(SolrTestCaseJ4.params(DelegationTokenHttpSolrClient.DELEGATION_TOKEN_PARAM, "mydt"))
+        .build()) {
+      fail();
+    } catch(Exception ex) {
+      if (!ex.getMessage().equals("parameter "+ DelegationTokenHttpSolrClient.DELEGATION_TOKEN_PARAM +" is redefined.")) {
+        throw ex;
+      }
+    }
+  }
 }


[40/44] lucene-solr:jira/solr-9854: Squashed commit of branch 'feature/metrics', containing: SOLR-4735: Improve Solr metrics reporting SOLR-9812: Implement /admin/metrics API SOLR-9805: Use metrics-jvm library to instrument jvm internals

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

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/licenses/metrics-jetty-LICENSE-ASL.txt
----------------------------------------------------------------------
diff --git a/solr/licenses/metrics-jetty-LICENSE-ASL.txt b/solr/licenses/metrics-jetty-LICENSE-ASL.txt
new file mode 100644
index 0000000..ccb320c
--- /dev/null
+++ b/solr/licenses/metrics-jetty-LICENSE-ASL.txt
@@ -0,0 +1,203 @@
+
+                                 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 2010-2012 Coda Hale and Yammer, Inc.
+
+   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.
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/licenses/metrics-jetty-NOTICE.txt
----------------------------------------------------------------------
diff --git a/solr/licenses/metrics-jetty-NOTICE.txt b/solr/licenses/metrics-jetty-NOTICE.txt
new file mode 100644
index 0000000..b4c6298
--- /dev/null
+++ b/solr/licenses/metrics-jetty-NOTICE.txt
@@ -0,0 +1,12 @@
+Metrics
+Copyright 2010-2013 Coda Hale and Yammer, Inc.
+
+This product includes software developed by Coda Hale and Yammer, Inc.
+
+This product includes code derived from the JSR-166 project (ThreadLocalRandom, Striped64,
+LongAdder), which was released with the following comments:
+
+    Written by Doug Lea with assistance from members of JCP JSR-166
+    Expert Group and released to the public domain, as explained at
+    http://creativecommons.org/publicdomain/zero/1.0/
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/licenses/metrics-jetty9-3.1.2.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/metrics-jetty9-3.1.2.jar.sha1 b/solr/licenses/metrics-jetty9-3.1.2.jar.sha1
new file mode 100644
index 0000000..0722b0b
--- /dev/null
+++ b/solr/licenses/metrics-jetty9-3.1.2.jar.sha1
@@ -0,0 +1 @@
+7f2fe1039424ca687bea5d09ec0bfa372bf7d062

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/licenses/metrics-json-LICENSE-ASL.txt
----------------------------------------------------------------------
diff --git a/solr/licenses/metrics-json-LICENSE-ASL.txt b/solr/licenses/metrics-json-LICENSE-ASL.txt
new file mode 100644
index 0000000..ccb320c
--- /dev/null
+++ b/solr/licenses/metrics-json-LICENSE-ASL.txt
@@ -0,0 +1,203 @@
+
+                                 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 2010-2012 Coda Hale and Yammer, Inc.
+
+   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.
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/licenses/metrics-json-NOTICE.txt
----------------------------------------------------------------------
diff --git a/solr/licenses/metrics-json-NOTICE.txt b/solr/licenses/metrics-json-NOTICE.txt
new file mode 100644
index 0000000..b4c6298
--- /dev/null
+++ b/solr/licenses/metrics-json-NOTICE.txt
@@ -0,0 +1,12 @@
+Metrics
+Copyright 2010-2013 Coda Hale and Yammer, Inc.
+
+This product includes software developed by Coda Hale and Yammer, Inc.
+
+This product includes code derived from the JSR-166 project (ThreadLocalRandom, Striped64,
+LongAdder), which was released with the following comments:
+
+    Written by Doug Lea with assistance from members of JCP JSR-166
+    Expert Group and released to the public domain, as explained at
+    http://creativecommons.org/publicdomain/zero/1.0/
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/licenses/metrics-jvm-3.1.2.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/metrics-jvm-3.1.2.jar.sha1 b/solr/licenses/metrics-jvm-3.1.2.jar.sha1
new file mode 100644
index 0000000..519fcdd
--- /dev/null
+++ b/solr/licenses/metrics-jvm-3.1.2.jar.sha1
@@ -0,0 +1 @@
+ed364e77218e50fdcdebce4d982cb4d1f4a8c187

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/licenses/metrics-jvm-LICENSE-ASL.txt
----------------------------------------------------------------------
diff --git a/solr/licenses/metrics-jvm-LICENSE-ASL.txt b/solr/licenses/metrics-jvm-LICENSE-ASL.txt
new file mode 100644
index 0000000..ccb320c
--- /dev/null
+++ b/solr/licenses/metrics-jvm-LICENSE-ASL.txt
@@ -0,0 +1,203 @@
+
+                                 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 2010-2012 Coda Hale and Yammer, Inc.
+
+   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.
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/licenses/metrics-jvm-NOTICE.txt
----------------------------------------------------------------------
diff --git a/solr/licenses/metrics-jvm-NOTICE.txt b/solr/licenses/metrics-jvm-NOTICE.txt
new file mode 100644
index 0000000..b4c6298
--- /dev/null
+++ b/solr/licenses/metrics-jvm-NOTICE.txt
@@ -0,0 +1,12 @@
+Metrics
+Copyright 2010-2013 Coda Hale and Yammer, Inc.
+
+This product includes software developed by Coda Hale and Yammer, Inc.
+
+This product includes code derived from the JSR-166 project (ThreadLocalRandom, Striped64,
+LongAdder), which was released with the following comments:
+
+    Written by Doug Lea with assistance from members of JCP JSR-166
+    Expert Group and released to the public domain, as explained at
+    http://creativecommons.org/publicdomain/zero/1.0/
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/licenses/metrics-servlets-LICENSE-ASL.txt
----------------------------------------------------------------------
diff --git a/solr/licenses/metrics-servlets-LICENSE-ASL.txt b/solr/licenses/metrics-servlets-LICENSE-ASL.txt
new file mode 100644
index 0000000..ccb320c
--- /dev/null
+++ b/solr/licenses/metrics-servlets-LICENSE-ASL.txt
@@ -0,0 +1,203 @@
+
+                                 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 2010-2012 Coda Hale and Yammer, Inc.
+
+   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.
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/licenses/metrics-servlets-NOTICE.txt
----------------------------------------------------------------------
diff --git a/solr/licenses/metrics-servlets-NOTICE.txt b/solr/licenses/metrics-servlets-NOTICE.txt
new file mode 100644
index 0000000..b4c6298
--- /dev/null
+++ b/solr/licenses/metrics-servlets-NOTICE.txt
@@ -0,0 +1,12 @@
+Metrics
+Copyright 2010-2013 Coda Hale and Yammer, Inc.
+
+This product includes software developed by Coda Hale and Yammer, Inc.
+
+This product includes code derived from the JSR-166 project (ThreadLocalRandom, Striped64,
+LongAdder), which was released with the following comments:
+
+    Written by Doug Lea with assistance from members of JCP JSR-166
+    Expert Group and released to the public domain, as explained at
+    http://creativecommons.org/publicdomain/zero/1.0/
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/server/build.xml
----------------------------------------------------------------------
diff --git a/solr/server/build.xml b/solr/server/build.xml
index 8600e17..ae4e650 100644
--- a/solr/server/build.xml
+++ b/solr/server/build.xml
@@ -39,8 +39,8 @@
   <target name="resolve" depends="ivy-availability-check,ivy-fail,ivy-configure">
     <sequential>
     <!-- jetty libs in lib/ -->
-    <ivy:retrieve conf="jetty,servlet" type="jar" log="download-only" symlink="${ivy.symlink}"
-                pattern="lib/[artifact]-[revision].[ext]" sync="${ivy.sync}"/>
+    <ivy:retrieve conf="jetty,servlet,metrics" type="jar,bundle" log="download-only" symlink="${ivy.symlink}"
+                  pattern="lib/[artifact]-[revision].[ext]" sync="${ivy.sync}"/>
     <ivy:retrieve conf="logging" type="jar,bundle" log="download-only" symlink="${ivy.symlink}"
                   pattern="lib/ext/[artifact]-[revision].[ext]" sync="${ivy.sync}"/>
     <!-- start.jar - we don't use sync=true here, we don't own the dir, but

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/server/etc/jetty.xml
----------------------------------------------------------------------
diff --git a/solr/server/etc/jetty.xml b/solr/server/etc/jetty.xml
index f1b94c8..8cb8223 100644
--- a/solr/server/etc/jetty.xml
+++ b/solr/server/etc/jetty.xml
@@ -29,9 +29,16 @@
   <!-- Consult the javadoc of o.e.j.util.thread.QueuedThreadPool   -->
   <!-- for all configuration that may be set here.                 -->
   <!-- =========================================================== -->
-  <!-- uncomment to change type of threadpool
-  <Arg name="threadpool"><New id="threadpool" class="org.eclipse.jetty.util.thread.QueuedThreadPool"/></Arg>
-  -->
+  <Arg name="threadpool">
+    <New id="threadpool" class="com.codahale.metrics.jetty9.InstrumentedQueuedThreadPool">
+      <Arg name="registry">
+        <Call id="solrJettyMetricRegistry" name="getOrCreate" class="com.codahale.metrics.SharedMetricRegistries">
+          <Arg>solr.jetty</Arg>
+        </Call>
+      </Arg>
+    </New>
+  </Arg>
+
   <Get name="ThreadPool">
     <Set name="minThreads" type="int"><Property name="solr.jetty.threads.min" default="10"/></Set>
     <Set name="maxThreads" type="int"><Property name="solr.jetty.threads.max" default="10000"/></Set>
@@ -106,7 +113,12 @@
              <New id="Contexts" class="org.eclipse.jetty.server.handler.ContextHandlerCollection"/>
            </Item>
            <Item>
-             <New id="DefaultHandler" class="org.eclipse.jetty.server.handler.DefaultHandler"/>
+             <New id="InstrumentedHandler" class="com.codahale.metrics.jetty9.InstrumentedHandler">
+               <Arg><Ref refid="solrJettyMetricRegistry"/></Arg>
+               <Set name="handler">
+                 <New id="DefaultHandler" class="org.eclipse.jetty.server.handler.DefaultHandler"/>
+               </Set>
+             </New>
            </Item>
            <Item>
              <New id="RequestLog" class="org.eclipse.jetty.server.handler.RequestLogHandler"/>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/server/ivy.xml
----------------------------------------------------------------------
diff --git a/solr/server/ivy.xml b/solr/server/ivy.xml
index 3a48224..8dc645a 100644
--- a/solr/server/ivy.xml
+++ b/solr/server/ivy.xml
@@ -18,7 +18,8 @@
 -->
 <ivy-module version="2.0">
   <info organisation="org.apache.solr" module="server"/>
-  <configurations defaultconfmapping="jetty->master;start->master;servlet->master;logging->master">
+  <configurations defaultconfmapping="metrics->master;jetty->master;start->master;servlet->master;logging->master">
+    <conf name="metrics" description="metrics jars" transitive="true"/>
     <conf name="jetty" description="jetty jars" transitive="false"/>
     <conf name="start" description="jetty start jar" transitive="false"/>
     <conf name="servlet" description="servlet-api jar" transitive="false"/>
@@ -30,8 +31,12 @@
     <dependency org="org.slf4j" name="slf4j-api" rev="${/org.slf4j/slf4j-api}" conf="logging"/>
     <dependency org="org.slf4j" name="jcl-over-slf4j" rev="${/org.slf4j/jcl-over-slf4j}" conf="logging"/>
     <dependency org="org.slf4j" name="jul-to-slf4j" rev="${/org.slf4j/jul-to-slf4j}" conf="logging"/> 
-    <dependency org="org.slf4j" name="slf4j-log4j12" rev="${/org.slf4j/slf4j-log4j12}" conf="logging"/> 
-      
+    <dependency org="org.slf4j" name="slf4j-log4j12" rev="${/org.slf4j/slf4j-log4j12}" conf="logging"/>
+
+    <dependency org="io.dropwizard.metrics" name="metrics-core" rev="${/io.dropwizard.metrics/metrics-core}" conf="metrics" />
+    <dependency org="io.dropwizard.metrics" name="metrics-jetty9" rev="${/io.dropwizard.metrics/metrics-jetty9}" conf="metrics" />
+    <dependency org="io.dropwizard.metrics" name="metrics-jvm" rev="${/io.dropwizard.metrics/metrics-jvm}" conf="metrics" />
+
     <dependency org="org.eclipse.jetty" name="jetty-continuation" rev="${/org.eclipse.jetty/jetty-continuation}" conf="jetty"/>
     <dependency org="org.eclipse.jetty" name="jetty-deploy" rev="${/org.eclipse.jetty/jetty-deploy}" conf="jetty"/>
     <dependency org="org.eclipse.jetty" name="jetty-http" rev="${/org.eclipse.jetty/jetty-http}" conf="jetty"/>

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java b/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
index 261d2ec..be8a24c 100644
--- a/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
+++ b/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
@@ -37,11 +37,13 @@ import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.core.CorePropertiesLocator;
 import org.apache.solr.core.CoresLocator;
 import org.apache.solr.core.NodeConfig;
+import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.SolrConfig;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.core.SolrXmlConfig;
 import org.apache.solr.handler.UpdateRequestHandler;
+import org.apache.solr.metrics.reporters.SolrJmxReporter;
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestHandler;
@@ -189,10 +191,17 @@ public class TestHarness extends BaseTestHarness {
         = new UpdateShardHandlerConfig(UpdateShardHandlerConfig.DEFAULT_MAXUPDATECONNECTIONS,
                                        UpdateShardHandlerConfig.DEFAULT_MAXUPDATECONNECTIONSPERHOST,
                                        30000, 30000);
+    // universal default metric reporter
+    Map<String,String> attributes = new HashMap<>();
+    attributes.put("name", "default");
+    attributes.put("class", SolrJmxReporter.class.getName());
+    PluginInfo defaultPlugin = new PluginInfo("reporter", attributes, null, null);
+
     return new NodeConfig.NodeConfigBuilder("testNode", loader)
         .setUseSchemaCache(Boolean.getBoolean("shareSchema"))
         .setCloudConfig(cloudConfig)
         .setUpdateShardHandlerConfig(updateShardHandlerConfig)
+        .setMetricReporterPlugins(new PluginInfo[] {defaultPlugin})
         .build();
   }
 


[23/44] lucene-solr:jira/solr-9854: LUCENE-7590: add DocValuesStats for SortedNumeric DV fields

Posted by ab...@apache.org.
LUCENE-7590: add DocValuesStats for SortedNumeric DV fields


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

Branch: refs/heads/jira/solr-9854
Commit: 944b8e07f557b9320895998fe33d71cae5199eee
Parents: dcf202a
Author: Shai Erera <sh...@apache.org>
Authored: Sat Dec 17 21:17:14 2016 +0200
Committer: Shai Erera <sh...@apache.org>
Committed: Sun Dec 18 08:39:42 2016 +0200

----------------------------------------------------------------------
 .../apache/lucene/search/DocValuesStats.java    | 128 +++++++++++++++-
 .../search/TestDocValuesStatsCollector.java     | 153 ++++++++++++++++++-
 2 files changed, 271 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/944b8e07/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java b/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
index c8b7752..9dd97a6 100644
--- a/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
+++ b/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
@@ -20,6 +20,7 @@ import java.io.IOException;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.SortedNumericDocValues;
 
 /** Holds statistics for a DocValues field. */
 public abstract class DocValuesStats<T> {
@@ -95,7 +96,7 @@ public abstract class DocValuesStats<T> {
   }
 
   /** Holds statistics for a numeric DocValues field. */
-  public static abstract class NumericDocValuesStats<T extends Number> extends DocValuesStats<T> {
+  static abstract class NumericDocValuesStats<T extends Number> extends DocValuesStats<T> {
 
     protected double mean = 0.0;
     protected double variance = 0.0;
@@ -113,7 +114,7 @@ public abstract class DocValuesStats<T> {
     }
 
     @Override
-    protected boolean hasValue(int doc) throws IOException {
+    protected final boolean hasValue(int doc) throws IOException {
       return ndv.advanceExact(doc);
     }
 
@@ -199,4 +200,127 @@ public abstract class DocValuesStats<T> {
     }
   }
 
+  /** Holds statistics for a sorted-numeric DocValues field. */
+  static abstract class SortedNumericDocValuesStats<T extends Number> extends DocValuesStats<T> {
+
+    protected long valuesCount = 0;
+    protected double mean = 0.0;
+    protected double variance = 0.0;
+
+    protected SortedNumericDocValues sndv;
+
+    protected SortedNumericDocValuesStats(String field, T initialMin, T initialMax) {
+      super(field, initialMin, initialMax);
+    }
+
+    @Override
+    protected final boolean init(LeafReaderContext context) throws IOException {
+      sndv = context.reader().getSortedNumericDocValues(field);
+      return sndv != null;
+    }
+
+    @Override
+    protected final boolean hasValue(int doc) throws IOException {
+      return sndv.advanceExact(doc);
+    }
+
+    /** The mean of all values of the field. */
+    public final double mean() {
+      return mean;
+    }
+
+    /** Returns the variance of all values of the field. */
+    public final double variance() {
+      int count = count();
+      return count > 0 ? variance / count : 0;
+    }
+
+    /** Returns the stdev of all values of the field. */
+    public final double stdev() {
+      return Math.sqrt(variance());
+    }
+
+    /** Returns the total number of values for this field. */
+    public final long valuesCount() {
+      return valuesCount;
+    }
+
+    /** Returns the sum of values of the field. Note that if the values are large, the {@code sum} might overflow. */
+    public abstract T sum();
+  }
+
+  /** Holds DocValues statistics for a sorted-numeric field storing {@code long} values. */
+  public static final class SortedLongDocValuesStats extends SortedNumericDocValuesStats<Long> {
+
+    // To avoid boxing 'long' to 'Long' while the sum is computed, declare it as private variable.
+    private long sum = 0;
+
+    public SortedLongDocValuesStats(String field) {
+      super(field, Long.MAX_VALUE, Long.MIN_VALUE);
+    }
+
+    @Override
+    protected void doAccumulate(int count) throws IOException {
+      int numValues = sndv.docValueCount();
+      while (numValues-- > 0) {
+        long val = sndv.nextValue();
+        if (val > max) {
+          max = val;
+        }
+        if (val < min) {
+          min = val;
+        }
+        sum += val;
+        double oldMean = mean;
+        // for correct "running average computation", increase valuesCount with each value, rather than once before the
+        // loop stats.
+        ++valuesCount;
+        mean += (val - mean) / valuesCount;
+        variance += (val - mean) * (val - oldMean);
+      }
+    }
+
+    @Override
+    public Long sum() {
+      return sum;
+    }
+  }
+
+  /** Holds DocValues statistics for a sorted-numeric field storing {@code double} values. */
+  public static final class SortedDoubleDocValuesStats extends SortedNumericDocValuesStats<Double> {
+
+    // To avoid boxing 'double' to 'Double' while the sum is computed, declare it as private variable.
+    private double sum = 0;
+
+    public SortedDoubleDocValuesStats(String field) {
+      super(field, Double.MAX_VALUE, Double.MIN_VALUE);
+    }
+
+    @Override
+    protected void doAccumulate(int count) throws IOException {
+      int numValues = sndv.docValueCount();
+      while (numValues-- > 0) {
+        double val = Double.longBitsToDouble(sndv.nextValue());
+        if (Double.compare(val, max) > 0) {
+          max = val;
+        }
+        if (Double.compare(val, min) < 0) {
+          min = val;
+        }
+        sum += val;
+        double oldMean = mean;
+        // for correct "running average computation", increase valuesCount with each value, rather than once before the
+        // loop stats.
+        ++valuesCount;
+        mean += (val - mean) / valuesCount;
+        variance += (val - mean) * (val - oldMean);
+      }
+    }
+
+    @Override
+    public Double sum() {
+      return sum;
+    }
+  }
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/944b8e07/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java b/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
index 8f8b09e..5fa4b04 100644
--- a/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
+++ b/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
@@ -20,19 +20,24 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.DoubleSummaryStatistics;
 import java.util.LongSummaryStatistics;
+import java.util.function.Predicate;
 import java.util.stream.DoubleStream;
 import java.util.stream.LongStream;
+import java.util.stream.Stream;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.DoubleDocValuesField;
 import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.SortedNumericDocValuesField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.DocValuesStats.DoubleDocValuesStats;
 import org.apache.lucene.search.DocValuesStats.LongDocValuesStats;
+import org.apache.lucene.search.DocValuesStats.SortedDoubleDocValuesStats;
+import org.apache.lucene.search.DocValuesStats.SortedLongDocValuesStats;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
@@ -185,20 +190,136 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
     }
   }
 
-  private static LongStream getPositiveValues(long[] docValues) {
-    return Arrays.stream(docValues).filter(v -> v > 0);
+  public void testDocsWithMultipleLongValues() throws IOException {
+    try (Directory dir = newDirectory();
+        IndexWriter indexWriter = new IndexWriter(dir, newIndexWriterConfig())) {
+      String field = "numeric";
+      int numDocs = TestUtil.nextInt(random(), 1, 100);
+      long[][] docValues = new long[numDocs][];
+      long nextVal = 1;
+      for (int i = 0; i < numDocs; i++) {
+        Document doc = new Document();
+        if (random().nextBoolean()) { // not all documents have a value
+          int numValues = TestUtil.nextInt(random(), 1, 5);
+          docValues[i] = new long[numValues];
+          for (int j = 0; j < numValues; j++) {
+            doc.add(new SortedNumericDocValuesField(field, nextVal));
+            docValues[i][j] = nextVal;
+            ++nextVal;
+          }
+          doc.add(new StringField("id", "doc" + i, Store.NO));
+        }
+        indexWriter.addDocument(doc);
+      }
+
+      // 20% of cases delete some docs
+      if (random().nextDouble() < 0.2) {
+        for (int i = 0; i < numDocs; i++) {
+          if (random().nextBoolean()) {
+            indexWriter.deleteDocuments(new Term("id", "doc" + i));
+            docValues[i] = null;
+          }
+        }
+      }
+
+      try (DirectoryReader reader = DirectoryReader.open(indexWriter)) {
+        IndexSearcher searcher = new IndexSearcher(reader);
+        SortedLongDocValuesStats stats = new SortedLongDocValuesStats(field);
+        searcher.search(new MatchAllDocsQuery(), new DocValuesStatsCollector(stats));
+
+        assertEquals(filterValues(docValues, (v) -> v != null).count(), stats.count());
+        assertEquals(filterValues(docValues, (v) -> v == null).count() - reader.numDeletedDocs(), stats.missing());
+        if (stats.count() > 0) {
+          LongSummaryStatistics sumStats = filterAndFlatValues(docValues, (v) -> v != null).summaryStatistics();
+          assertEquals(sumStats.getMax(), stats.max().longValue());
+          assertEquals(sumStats.getMin(), stats.min().longValue());
+          assertEquals(sumStats.getAverage(), stats.mean(), 0.00001);
+          assertEquals(sumStats.getSum(), stats.sum().longValue());
+          assertEquals(sumStats.getCount(), stats.valuesCount());
+          double variance = computeVariance(filterAndFlatValues(docValues, (v) -> v != null), stats.mean, stats.count());
+          assertEquals(variance, stats.variance(), 0.00001);
+          assertEquals(Math.sqrt(variance), stats.stdev(), 0.00001);
+        }
+      }
+    }
   }
 
-  private static DoubleStream getPositiveValues(double[] docValues) {
-    return Arrays.stream(docValues).filter(v -> v > 0);
+  public void testDocsWithMultipleDoubleValues() throws IOException {
+    try (Directory dir = newDirectory();
+        IndexWriter indexWriter = new IndexWriter(dir, newIndexWriterConfig())) {
+      String field = "numeric";
+      int numDocs = TestUtil.nextInt(random(), 1, 100);
+      double[][] docValues = new double[numDocs][];
+      double nextVal = 1;
+      for (int i = 0; i < numDocs; i++) {
+        Document doc = new Document();
+        if (random().nextBoolean()) { // not all documents have a value
+          int numValues = TestUtil.nextInt(random(), 1, 5);
+          docValues[i] = new double[numValues];
+          for (int j = 0; j < numValues; j++) {
+            doc.add(new SortedNumericDocValuesField(field, Double.doubleToRawLongBits(nextVal)));
+            docValues[i][j] = nextVal;
+            ++nextVal;
+          }
+          doc.add(new StringField("id", "doc" + i, Store.NO));
+        }
+        indexWriter.addDocument(doc);
+      }
+
+      // 20% of cases delete some docs
+      if (random().nextDouble() < 0.2) {
+        for (int i = 0; i < numDocs; i++) {
+          if (random().nextBoolean()) {
+            indexWriter.deleteDocuments(new Term("id", "doc" + i));
+            docValues[i] = null;
+          }
+        }
+      }
+
+      try (DirectoryReader reader = DirectoryReader.open(indexWriter)) {
+        IndexSearcher searcher = new IndexSearcher(reader);
+        SortedDoubleDocValuesStats stats = new SortedDoubleDocValuesStats(field);
+        searcher.search(new MatchAllDocsQuery(), new DocValuesStatsCollector(stats));
+
+        assertEquals(filterValues(docValues, (v) -> v != null).count(), stats.count());
+        assertEquals(filterValues(docValues, (v) -> v == null).count() - reader.numDeletedDocs(), stats.missing());
+        if (stats.count() > 0) {
+          DoubleSummaryStatistics sumStats = filterAndFlatValues(docValues, (v) -> v != null).summaryStatistics();
+          assertEquals(sumStats.getMax(), stats.max().longValue(), 0.00001);
+          assertEquals(sumStats.getMin(), stats.min().longValue(), 0.00001);
+          assertEquals(sumStats.getAverage(), stats.mean(), 0.00001);
+          assertEquals(sumStats.getSum(), stats.sum().doubleValue(), 0.00001);
+          assertEquals(sumStats.getCount(), stats.valuesCount());
+          double variance = computeVariance(filterAndFlatValues(docValues, (v) -> v != null), stats.mean, stats.count());
+          assertEquals(variance, stats.variance(), 0.00001);
+          assertEquals(Math.sqrt(variance), stats.stdev(), 0.00001);
+        }
+      }
+    }
+  }
+
+  private static LongStream getPositiveValues(long[] values) {
+    return Arrays.stream(values).filter(v -> v > 0);
+  }
+
+  private static DoubleStream getPositiveValues(double[] values) {
+    return Arrays.stream(values).filter(v -> v > 0);
+  }
+
+  private static LongStream getZeroValues(long[] values) {
+    return Arrays.stream(values).filter(v -> v == 0);
   }
 
-  private static LongStream getZeroValues(long[] docValues) {
-    return Arrays.stream(docValues).filter(v -> v == 0);
+  private static DoubleStream getZeroValues(double[] values) {
+    return Arrays.stream(values).filter(v -> v == 0);
   }
 
-  private static DoubleStream getZeroValues(double[] docValues) {
-    return Arrays.stream(docValues).filter(v -> v == 0);
+  private static Stream<long[]> filterValues(long[][] values, Predicate<? super long[]> p) {
+    return Arrays.stream(values).filter(p);
+  }
+
+  private static Stream<double[]> filterValues(double[][] values, Predicate<? super double[]> p) {
+    return Arrays.stream(values).filter(p);
   }
 
   private static double computeVariance(long[] values, double mean, int count) {
@@ -209,4 +330,20 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
     return getPositiveValues(values).map(v -> (v - mean) * (v-mean)).sum() / count;
   }
 
+  private static LongStream filterAndFlatValues(long[][] values, Predicate<? super long[]> p) {
+    return filterValues(values, (v) -> v != null).flatMapToLong(Arrays::stream);
+  }
+
+  private static DoubleStream filterAndFlatValues(double[][] values, Predicate<? super double[]> p) {
+    return filterValues(values, (v) -> v != null).flatMapToDouble(Arrays::stream);
+  }
+
+  private static double computeVariance(LongStream values, double mean, int count) {
+    return values.mapToDouble(v -> (v - mean) * (v-mean)).sum() / count;
+  }
+
+  private static double computeVariance(DoubleStream values, double mean, int count) {
+    return values.map(v -> (v - mean) * (v-mean)).sum() / count;
+  }
+
 }


[26/44] lucene-solr:jira/solr-9854: SOLR-4735 Add gauges that report the current number of cores.

Posted by ab...@apache.org.
SOLR-4735 Add gauges that report the current number of cores.


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

Branch: refs/heads/jira/solr-9854
Commit: 92ef10dbdea6d8764d2a1ecba3d53abee542542d
Parents: 5a17c1b
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Mon Dec 19 13:34:15 2016 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Mon Dec 19 13:34:15 2016 +0100

----------------------------------------------------------------------
 .../src/java/org/apache/solr/core/CoreContainer.java   | 13 +++++++++++++
 .../org/apache/solr/metrics/SolrMetricManager.java     |  8 ++++----
 .../apache/solr/handler/admin/MetricsHandlerTest.java  |  2 ++
 3 files changed, 19 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/92ef10db/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 c423985..0703212 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -33,6 +33,7 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
 
+import com.codahale.metrics.Gauge;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
 import org.apache.http.auth.AuthSchemeProvider;
@@ -491,6 +492,18 @@ public class CoreContainer {
 
     containerProperties.putAll(cfg.getSolrProperties());
 
+    // initialize gauges for reporting the number of cores
+    Gauge<Integer> loadedCores = () -> solrCores.getCores().size();
+    Gauge<Integer> lazyCores = () -> solrCores.getCoreNames().size() - solrCores.getCores().size();
+    Gauge<Integer> unloadedCores = () -> solrCores.getAllCoreNames().size() - solrCores.getCoreNames().size();
+
+    SolrMetricManager.register(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node),
+        loadedCores, true, "loaded", "cores");
+    SolrMetricManager.register(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node),
+        lazyCores, true, "lazy", "cores");
+    SolrMetricManager.register(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node),
+        unloadedCores, true, "unloaded", "cores");
+
     // setup executor to load cores in parallel
     ExecutorService coreLoadExecutor = ExecutorUtil.newMDCAwareFixedThreadPool(
         cfg.getCoreLoadThreadCount(isZooKeeperAware()),

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/92ef10db/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java b/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
index 2063689..b700782 100644
--- a/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
@@ -280,18 +280,18 @@ public class SolrMetricManager {
    * Register an instance of {@link Metric}.
    * @param registry registry name
    * @param metric metric instance
-   * @param skipExisting if true then already an existing metric with the same name will be kept.
+   * @param force if true then an already existing metric with the same name will be replaced.
    *                     When false and a metric with the same name already exists an exception
    *                     will be thrown.
    * @param metricName metric name, either final name or a fully-qualified name
    *                   using dotted notation
    * @param metricPath (optional) additional top-most metric name path elements
    */
-  public static void register(String registry, Metric metric, boolean skipExisting, String metricName, String... metricPath) {
+  public static void register(String registry, Metric metric, boolean force, String metricName, String... metricPath) {
     MetricRegistry metricRegistry = registry(registry);
     String fullName = mkName(metricName, metricPath);
-    if (skipExisting && metricRegistry.getMetrics().containsKey(fullName)) {
-      return;
+    if (force && metricRegistry.getMetrics().containsKey(fullName)) {
+      metricRegistry.remove(fullName);
     }
     metricRegistry.register(fullName, metric);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/92ef10db/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 83ce74f..3667285 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
@@ -53,6 +53,8 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 {
     assertNotNull(((NamedList) nl.get("newSearcherErrors")).get("requests"));
     assertEquals(0L, ((NamedList) nl.get("newSearcherErrors")).get("requests"));
     nl = (NamedList) values.get("solr.node");
+    assertNotNull(nl.get("cores.loaded")); // int gauge
+    assertEquals(1, ((NamedList) nl.get("cores.loaded")).get("value"));
     assertNotNull(nl.get("QUERYHANDLER./admin/authorization.clientErrors")); // timer type
     assertEquals(5, ((NamedList) nl.get("QUERYHANDLER./admin/authorization.clientErrors")).size());
 


[07/44] lucene-solr:jira/solr-9854: SOLR-9846: OverseerAutoReplicaFailoverThread can take too long to stop and leak out of unit tests.

Posted by ab...@apache.org.
SOLR-9846: OverseerAutoReplicaFailoverThread can take too long to stop and leak out of unit tests.


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

Branch: refs/heads/jira/solr-9854
Commit: 7dec783b287ab554cc781622b4d6127e553fd2ae
Parents: e82399d
Author: markrmiller <ma...@apache.org>
Authored: Sun Dec 11 22:02:48 2016 -0500
Committer: markrmiller <ma...@apache.org>
Committed: Wed Dec 14 12:57:21 2016 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                            | 2 ++
 .../solr/cloud/OverseerAutoReplicaFailoverThread.java       | 9 +++++++--
 2 files changed, 9 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7dec783b/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 946a04e..5f0357b 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -288,6 +288,8 @@ Other Changes
   response (instead of a SolrException) and includes the remote error message as part of the exception message
   (Tom�s Fern�ndez L�bbe)
 
+* SOLR-9846: OverseerAutoReplicaFailoverThread can take too long to stop and leak out of unit tests. (Mark Miller)
+
 ==================  6.3.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7dec783b/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java b/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java
index 83679a5..10b4bf3 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java
@@ -89,6 +89,8 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable {
 
   private final int workLoopDelay;
   private final int waitAfterExpiration;
+
+  private volatile Thread thread;
   
   public OverseerAutoReplicaFailoverThread(CloudConfig config, ZkStateReader zkStateReader,
       UpdateShardHandler updateShardHandler) {
@@ -118,7 +120,7 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable {
   
   @Override
   public void run() {
-    
+    this.thread = Thread.currentThread();
     while (!this.isClosed) {
       // work loop
       log.debug("do " + this.getClass().getSimpleName() + " work loop");
@@ -136,7 +138,6 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable {
         try {
           Thread.sleep(workLoopDelay);
         } catch (InterruptedException e) {
-          Thread.currentThread().interrupt();
           return;
         }
       }
@@ -480,6 +481,10 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable {
   @Override
   public void close() {
     isClosed = true;
+    Thread lThread = thread;
+    if (lThread != null) {
+      lThread.interrupt();
+    }
   }
   
   public boolean isClosed() {


[20/44] lucene-solr:jira/solr-9854: SOLR-9712: block when maxWarmingSearchers is exceeded instead of throwing exception, default to 1, remove from most configs

Posted by ab...@apache.org.
SOLR-9712: block when maxWarmingSearchers is exceeded instead of throwing exception, default to 1, remove from most configs


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

Branch: refs/heads/jira/solr-9854
Commit: c9522a393661c8878d488ad4475ac7e2cbb9c25c
Parents: 835296f
Author: yonik <yo...@apache.org>
Authored: Fri Dec 16 11:46:48 2016 -0500
Committer: yonik <yo...@apache.org>
Committed: Fri Dec 16 11:46:48 2016 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  6 ++
 .../solr/collection1/conf/solrconfig.xml        |  5 --
 solr/contrib/ltr/example/solrconfig.xml         | 11 ---
 .../solr/solrcloud/conf/solrconfig.xml          | 11 ---
 .../java/org/apache/solr/core/SolrConfig.java   |  2 +-
 .../src/java/org/apache/solr/core/SolrCore.java | 84 +++++++++++---------
 .../solr/collection1/conf/solrconfig_perf.xml   |  1 -
 .../org/apache/solr/search/TestRealTimeGet.java |  2 +-
 .../apache/solr/search/TestStressLucene.java    |  2 +-
 .../apache/solr/search/TestStressRecovery.java  |  2 +-
 .../apache/solr/search/TestStressReorder.java   |  5 +-
 .../solr/search/TestStressUserVersions.java     |  4 +-
 .../apache/solr/search/TestStressVersions.java  |  2 +-
 .../example-DIH/solr/db/conf/solrconfig.xml     | 11 ---
 .../example-DIH/solr/mail/conf/solrconfig.xml   | 11 ---
 .../example-DIH/solr/rss/conf/solrconfig.xml    | 11 ---
 .../example-DIH/solr/solr/conf/solrconfig.xml   | 11 ---
 .../example-DIH/solr/tika/conf/solrconfig.xml   | 11 ---
 solr/example/files/conf/solrconfig.xml          | 11 ---
 .../basic_configs/conf/solrconfig.xml           | 11 ---
 .../conf/solrconfig.xml                         | 11 ---
 .../conf/solrconfig.xml                         | 11 ---
 .../solr/client/solrj/LargeVolumeTestBase.java  |  3 +-
 .../solr/client/solrj/SolrExampleTestBase.java  |  1 -
 .../java/org/apache/solr/SolrJettyTestBase.java |  2 -
 25 files changed, 62 insertions(+), 180 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 73b0e9b..c920575 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -114,6 +114,12 @@ Upgrade Notes
   for a listing of highlight parameters annotated with which highlighters use them.
   hl.useFastVectorHighlighter is now considered deprecated in lieu of hl.method=fastVector.
 
+* SOLR-9712: maxWarmingSearchers now defaults to 1, and more importantly commits will now block if this
+  limit is exceeded instead of throwing an exception (a good thing). Consequently there is no longer a
+  risk in overlapping commits.  Nonetheless users should continue to avoid excessive committing. Users are
+  advised to remove any pre-existing maxWarmingSearchers entries from their solrconfig.xml files.
+
+
 New Features
 ----------------------
 * SOLR-9293: Solrj client support for hierarchical clusters and other topics 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/contrib/clustering/src/test-files/clustering/solr/collection1/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/contrib/clustering/src/test-files/clustering/solr/collection1/conf/solrconfig.xml b/solr/contrib/clustering/src/test-files/clustering/solr/collection1/conf/solrconfig.xml
index 5e0e66c..6482286 100644
--- a/solr/contrib/clustering/src/test-files/clustering/solr/collection1/conf/solrconfig.xml
+++ b/solr/contrib/clustering/src/test-files/clustering/solr/collection1/conf/solrconfig.xml
@@ -212,11 +212,6 @@
          warming. -->
     <useColdSearcher>false</useColdSearcher>
 
-    <!-- Maximum number of searchers that may be warming in the background
-      concurrently.  An error is returned if this limit is exceeded. Recommend
-      1-2 for read-only slaves, higher for masters w/o cache warming. -->
-    <maxWarmingSearchers>2</maxWarmingSearchers>
-
   </query>
 
   <!-- 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/contrib/ltr/example/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/example/solrconfig.xml b/solr/contrib/ltr/example/solrconfig.xml
index 55da6a4..1c66c49 100644
--- a/solr/contrib/ltr/example/solrconfig.xml
+++ b/solr/contrib/ltr/example/solrconfig.xml
@@ -609,17 +609,6 @@
       -->
     <useColdSearcher>false</useColdSearcher>
 
-    <!-- Max Warming Searchers
-
-         Maximum number of searchers that may be warming in the
-         background concurrently.  An error is returned if this limit
-         is exceeded.
-
-         Recommend values of 1-2 for read-only slaves, higher for
-         masters w/o cache warming.
-      -->
-    <maxWarmingSearchers>2</maxWarmingSearchers>
-
   </query>
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/contrib/morphlines-core/src/test-files/solr/solrcloud/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcloud/conf/solrconfig.xml b/solr/contrib/morphlines-core/src/test-files/solr/solrcloud/conf/solrconfig.xml
index 1eda53e..42ab056 100644
--- a/solr/contrib/morphlines-core/src/test-files/solr/solrcloud/conf/solrconfig.xml
+++ b/solr/contrib/morphlines-core/src/test-files/solr/solrcloud/conf/solrconfig.xml
@@ -606,17 +606,6 @@
       -->
     <useColdSearcher>false</useColdSearcher>
 
-    <!-- Max Warming Searchers
-         
-         Maximum number of searchers that may be warming in the
-         background concurrently.  An error is returned if this limit
-         is exceeded.
-
-         Recommend values of 1-2 for read-only slaves, higher for
-         masters w/o cache warming.
-      -->
-    <maxWarmingSearchers>4</maxWarmingSearchers>
-
   </query>
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/core/src/java/org/apache/solr/core/SolrConfig.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrConfig.java b/solr/core/src/java/org/apache/solr/core/SolrConfig.java
index e5e25a1..bd98075 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrConfig.java
@@ -287,7 +287,7 @@ public class SolrConfig extends Config implements MapSerializable {
     } else {
       jmxConfig = new JmxConfiguration(false, null, null, null);
     }
-    maxWarmingSearchers = getInt("query/maxWarmingSearchers", Integer.MAX_VALUE);
+    maxWarmingSearchers = getInt("query/maxWarmingSearchers", 1);
     slowQueryThresholdMillis = getInt("query/slowQueryThresholdMillis", -1);
     for (SolrPluginInfo plugin : plugins) loadPluginInfo(plugin);
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/core/src/java/org/apache/solr/core/SolrCore.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrCore.java b/solr/core/src/java/org/apache/solr/core/SolrCore.java
index a459bf2..b3facef 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -1886,51 +1886,57 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
     // if it isn't necessary.
 
     synchronized (searcherLock) {
-      // see if we can return the current searcher
-      if (_searcher!=null && !forceNew) {
-        if (returnSearcher) {
-          _searcher.incref();
-          return _searcher;
-        } else {
-          return null;
+      for(;;) { // this loop is so w can retry in the event that we exceed maxWarmingSearchers
+        // see if we can return the current searcher
+        if (_searcher != null && !forceNew) {
+          if (returnSearcher) {
+            _searcher.incref();
+            return _searcher;
+          } else {
+            return null;
+          }
         }
-      }
 
-      // check to see if we can wait for someone else's searcher to be set
-      if (onDeckSearchers>0 && !forceNew && _searcher==null) {
-        try {
-          searcherLock.wait();
-        } catch (InterruptedException e) {
-          log.info(SolrException.toStr(e));
+        // check to see if we can wait for someone else's searcher to be set
+        if (onDeckSearchers > 0 && !forceNew && _searcher == null) {
+          try {
+            searcherLock.wait();
+          } catch (InterruptedException e) {
+            log.info(SolrException.toStr(e));
+          }
         }
-      }
 
-      // check again: see if we can return right now
-      if (_searcher!=null && !forceNew) {
-        if (returnSearcher) {
-          _searcher.incref();
-          return _searcher;
-        } else {
-          return null;
+        // check again: see if we can return right now
+        if (_searcher != null && !forceNew) {
+          if (returnSearcher) {
+            _searcher.incref();
+            return _searcher;
+          } else {
+            return null;
+          }
         }
-      }
 
-      // At this point, we know we need to open a new searcher...
-      // first: increment count to signal other threads that we are
-      //        opening a new searcher.
-      onDeckSearchers++;
-      if (onDeckSearchers < 1) {
-        // should never happen... just a sanity check
-        log.error(logid+"ERROR!!! onDeckSearchers is " + onDeckSearchers);
-        onDeckSearchers=1;  // reset
-      } else if (onDeckSearchers > maxWarmingSearchers) {
-        onDeckSearchers--;
-        String msg="Error opening new searcher. exceeded limit of maxWarmingSearchers="+maxWarmingSearchers + ", try again later.";
-        log.warn(logid+""+ msg);
-        // HTTP 503==service unavailable, or 409==Conflict
-        throw new SolrException(SolrException.ErrorCode.SERVICE_UNAVAILABLE,msg);
-      } else if (onDeckSearchers > 1) {
-        log.warn(logid+"PERFORMANCE WARNING: Overlapping onDeckSearchers=" + onDeckSearchers);
+        // At this point, we know we need to open a new searcher...
+        // first: increment count to signal other threads that we are
+        //        opening a new searcher.
+        onDeckSearchers++;
+        if (onDeckSearchers < 1) {
+          // should never happen... just a sanity check
+          log.error(logid + "ERROR!!! onDeckSearchers is " + onDeckSearchers);
+          onDeckSearchers = 1;  // reset
+        } else if (onDeckSearchers > maxWarmingSearchers) {
+          onDeckSearchers--;
+          try {
+            searcherLock.wait();
+          } catch (InterruptedException e) {
+            log.info(SolrException.toStr(e));
+          }
+          continue;  // go back to the top of the loop and retry
+        } else if (onDeckSearchers > 1) {
+          log.warn(logid + "PERFORMANCE WARNING: Overlapping onDeckSearchers=" + onDeckSearchers);
+        }
+
+        break; // I can now exit the loop and proceed to open a searcher
       }
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/core/src/test-files/solr/collection1/conf/solrconfig_perf.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig_perf.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig_perf.xml
index 8bd2dbb..c0b297f 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig_perf.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig_perf.xml
@@ -57,7 +57,6 @@
     <queryResultWindowSize>20</queryResultWindowSize>
     <queryResultMaxDocsCached>200</queryResultMaxDocsCached>
     <useColdSearcher>false</useColdSearcher>
-    <maxWarmingSearchers>2</maxWarmingSearchers>
 
   </query>
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/core/src/test/org/apache/solr/search/TestRealTimeGet.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestRealTimeGet.java b/solr/core/src/test/org/apache/solr/search/TestRealTimeGet.java
index 51eaf34..28ecaa3 100644
--- a/solr/core/src/test/org/apache/solr/search/TestRealTimeGet.java
+++ b/solr/core/src/test/org/apache/solr/search/TestRealTimeGet.java
@@ -513,7 +513,7 @@ public class TestRealTimeGet extends TestRTGBase {
     final int ndocs = 5 + (random().nextBoolean() ? random().nextInt(25) : random().nextInt(200));
     int nWriteThreads = 5 + random().nextInt(25);
 
-    final int maxConcurrentCommits = nWriteThreads;   // number of committers at a time... it should be <= maxWarmingSearchers
+    final int maxConcurrentCommits = nWriteThreads;   // number of committers at a time...
 
         // query variables
     final int percentRealtimeQuery = 60;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/core/src/test/org/apache/solr/search/TestStressLucene.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestStressLucene.java b/solr/core/src/test/org/apache/solr/search/TestStressLucene.java
index 94cf300..779be43 100644
--- a/solr/core/src/test/org/apache/solr/search/TestStressLucene.java
+++ b/solr/core/src/test/org/apache/solr/search/TestStressLucene.java
@@ -55,7 +55,7 @@ public class TestStressLucene extends TestRTGBase {
     final int ndocs = 5 + (random().nextBoolean() ? random().nextInt(25) : random().nextInt(200));
     int nWriteThreads = 5 + random().nextInt(25);
 
-    final int maxConcurrentCommits = nWriteThreads;   // number of committers at a time... it should be <= maxWarmingSearchers
+    final int maxConcurrentCommits = nWriteThreads;
 
     final AtomicLong operations = new AtomicLong(100000);  // number of query operations to perform in total
     int nReadThreads = 5 + random().nextInt(25);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/core/src/test/org/apache/solr/search/TestStressRecovery.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestStressRecovery.java b/solr/core/src/test/org/apache/solr/search/TestStressRecovery.java
index 412908d..b6ecc2e 100644
--- a/solr/core/src/test/org/apache/solr/search/TestStressRecovery.java
+++ b/solr/core/src/test/org/apache/solr/search/TestStressRecovery.java
@@ -70,7 +70,7 @@ public class TestStressRecovery extends TestRTGBase {
     final int ndocs = 5 + (random().nextBoolean() ? random().nextInt(25) : random().nextInt(200));
     int nWriteThreads = 2 + random().nextInt(10);  // fewer write threads to give recovery thread more of a chance
 
-    final int maxConcurrentCommits = nWriteThreads;   // number of committers at a time... it should be <= maxWarmingSearchers
+    final int maxConcurrentCommits = nWriteThreads;
 
     // query variables
     final int percentRealtimeQuery = 75;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/core/src/test/org/apache/solr/search/TestStressReorder.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestStressReorder.java b/solr/core/src/test/org/apache/solr/search/TestStressReorder.java
index 4415946..bce1d75 100644
--- a/solr/core/src/test/org/apache/solr/search/TestStressReorder.java
+++ b/solr/core/src/test/org/apache/solr/search/TestStressReorder.java
@@ -68,8 +68,7 @@ public class TestStressReorder extends TestRTGBase {
     final int ndocs = 5 + (random().nextBoolean() ? random().nextInt(25) : random().nextInt(200));
     int nWriteThreads = 5 + random().nextInt(25);
 
-    final int maxConcurrentCommits = nWriteThreads;   // number of committers at a time... it should be <= maxWarmingSearchers
-
+    final int maxConcurrentCommits = nWriteThreads;
         // query variables
     final int percentRealtimeQuery = 75;
     final AtomicLong operations = new AtomicLong(50000);  // number of query operations to perform in total
@@ -84,7 +83,7 @@ public class TestStressReorder extends TestRTGBase {
     final int ndocs = 1;
     int nWriteThreads = 2;
 
-    final int maxConcurrentCommits = nWriteThreads;   // number of committers at a time... it should be <= maxWarmingSearchers
+    final int maxConcurrentCommits = nWriteThreads;
 
     // query variables
     final int percentRealtimeQuery = 101;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/core/src/test/org/apache/solr/search/TestStressUserVersions.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestStressUserVersions.java b/solr/core/src/test/org/apache/solr/search/TestStressUserVersions.java
index e098ecc..4eea434 100644
--- a/solr/core/src/test/org/apache/solr/search/TestStressUserVersions.java
+++ b/solr/core/src/test/org/apache/solr/search/TestStressUserVersions.java
@@ -74,7 +74,7 @@ public class TestStressUserVersions extends TestRTGBase {
     final int ndocs = 5 + (random().nextBoolean() ? random().nextInt(25) : random().nextInt(200));
     int nWriteThreads = 5 + random().nextInt(25);
 
-    final int maxConcurrentCommits = nWriteThreads;   // number of committers at a time... it should be <= maxWarmingSearchers
+    final int maxConcurrentCommits = nWriteThreads;
 
     // query variables
     final int percentRealtimeQuery = 75;
@@ -90,7 +90,7 @@ public class TestStressUserVersions extends TestRTGBase {
      final int ndocs = 1;
      int nWriteThreads = 2;
 
-     final int maxConcurrentCommits = nWriteThreads;   // number of committers at a time... it should be <= maxWarmingSearchers
+     final int maxConcurrentCommits = nWriteThreads;
 
      // query variables
      final int percentRealtimeQuery = 101;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/core/src/test/org/apache/solr/search/TestStressVersions.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestStressVersions.java b/solr/core/src/test/org/apache/solr/search/TestStressVersions.java
index d91a2cc..ed51ae5 100644
--- a/solr/core/src/test/org/apache/solr/search/TestStressVersions.java
+++ b/solr/core/src/test/org/apache/solr/search/TestStressVersions.java
@@ -56,7 +56,7 @@ public class TestStressVersions extends TestRTGBase {
     final int ndocs = 5 + (random().nextBoolean() ? random().nextInt(25) : random().nextInt(200));
     int nWriteThreads = 5 + random().nextInt(25);
 
-    final int maxConcurrentCommits = nWriteThreads;   // number of committers at a time... it should be <= maxWarmingSearchers
+    final int maxConcurrentCommits = nWriteThreads;
 
     // query variables
     final int percentRealtimeQuery = 75;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/example/example-DIH/solr/db/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/example/example-DIH/solr/db/conf/solrconfig.xml b/solr/example/example-DIH/solr/db/conf/solrconfig.xml
index 6a30473..71d8f5a 100644
--- a/solr/example/example-DIH/solr/db/conf/solrconfig.xml
+++ b/solr/example/example-DIH/solr/db/conf/solrconfig.xml
@@ -581,17 +581,6 @@
       -->
     <useColdSearcher>false</useColdSearcher>
 
-    <!-- Max Warming Searchers
-         
-         Maximum number of searchers that may be warming in the
-         background concurrently.  An error is returned if this limit
-         is exceeded.
-
-         Recommend values of 1-2 for read-only slaves, higher for
-         masters w/o cache warming.
-      -->
-    <maxWarmingSearchers>2</maxWarmingSearchers>
-
   </query>
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/example/example-DIH/solr/mail/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/example/example-DIH/solr/mail/conf/solrconfig.xml b/solr/example/example-DIH/solr/mail/conf/solrconfig.xml
index 24da135..8bc24f5 100644
--- a/solr/example/example-DIH/solr/mail/conf/solrconfig.xml
+++ b/solr/example/example-DIH/solr/mail/conf/solrconfig.xml
@@ -584,17 +584,6 @@
       -->
     <useColdSearcher>false</useColdSearcher>
 
-    <!-- Max Warming Searchers
-         
-         Maximum number of searchers that may be warming in the
-         background concurrently.  An error is returned if this limit
-         is exceeded.
-
-         Recommend values of 1-2 for read-only slaves, higher for
-         masters w/o cache warming.
-      -->
-    <maxWarmingSearchers>2</maxWarmingSearchers>
-
   </query>
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/example/example-DIH/solr/rss/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/example/example-DIH/solr/rss/conf/solrconfig.xml b/solr/example/example-DIH/solr/rss/conf/solrconfig.xml
index 224a94b..eae60a7 100644
--- a/solr/example/example-DIH/solr/rss/conf/solrconfig.xml
+++ b/solr/example/example-DIH/solr/rss/conf/solrconfig.xml
@@ -581,17 +581,6 @@
       -->
     <useColdSearcher>false</useColdSearcher>
 
-    <!-- Max Warming Searchers
-         
-         Maximum number of searchers that may be warming in the
-         background concurrently.  An error is returned if this limit
-         is exceeded.
-
-         Recommend values of 1-2 for read-only slaves, higher for
-         masters w/o cache warming.
-      -->
-    <maxWarmingSearchers>2</maxWarmingSearchers>
-
   </query>
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/example/example-DIH/solr/solr/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/example/example-DIH/solr/solr/conf/solrconfig.xml b/solr/example/example-DIH/solr/solr/conf/solrconfig.xml
index 2c67189..873391b 100644
--- a/solr/example/example-DIH/solr/solr/conf/solrconfig.xml
+++ b/solr/example/example-DIH/solr/solr/conf/solrconfig.xml
@@ -581,17 +581,6 @@
       -->
     <useColdSearcher>false</useColdSearcher>
 
-    <!-- Max Warming Searchers
-         
-         Maximum number of searchers that may be warming in the
-         background concurrently.  An error is returned if this limit
-         is exceeded.
-
-         Recommend values of 1-2 for read-only slaves, higher for
-         masters w/o cache warming.
-      -->
-    <maxWarmingSearchers>2</maxWarmingSearchers>
-
   </query>
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/example/example-DIH/solr/tika/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/example/example-DIH/solr/tika/conf/solrconfig.xml b/solr/example/example-DIH/solr/tika/conf/solrconfig.xml
index 7554e17..ac0c22a 100644
--- a/solr/example/example-DIH/solr/tika/conf/solrconfig.xml
+++ b/solr/example/example-DIH/solr/tika/conf/solrconfig.xml
@@ -584,17 +584,6 @@
       -->
     <useColdSearcher>false</useColdSearcher>
 
-    <!-- Max Warming Searchers
-         
-         Maximum number of searchers that may be warming in the
-         background concurrently.  An error is returned if this limit
-         is exceeded.
-
-         Recommend values of 1-2 for read-only slaves, higher for
-         masters w/o cache warming.
-      -->
-    <maxWarmingSearchers>2</maxWarmingSearchers>
-
   </query>
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/example/files/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/example/files/conf/solrconfig.xml b/solr/example/files/conf/solrconfig.xml
index fcbc4d6..3fd825e 100644
--- a/solr/example/files/conf/solrconfig.xml
+++ b/solr/example/files/conf/solrconfig.xml
@@ -579,17 +579,6 @@
       -->
     <useColdSearcher>false</useColdSearcher>
 
-    <!-- Max Warming Searchers
-         
-         Maximum number of searchers that may be warming in the
-         background concurrently.  An error is returned if this limit
-         is exceeded.
-
-         Recommend values of 1-2 for read-only slaves, higher for
-         masters w/o cache warming.
-      -->
-    <maxWarmingSearchers>2</maxWarmingSearchers>
-
   </query>
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/server/solr/configsets/basic_configs/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/server/solr/configsets/basic_configs/conf/solrconfig.xml b/solr/server/solr/configsets/basic_configs/conf/solrconfig.xml
index b0a8cdf..4ef902f 100644
--- a/solr/server/solr/configsets/basic_configs/conf/solrconfig.xml
+++ b/solr/server/solr/configsets/basic_configs/conf/solrconfig.xml
@@ -599,17 +599,6 @@
       -->
     <useColdSearcher>false</useColdSearcher>
 
-    <!-- Max Warming Searchers
-         
-         Maximum number of searchers that may be warming in the
-         background concurrently.  An error is returned if this limit
-         is exceeded.
-
-         Recommend values of 1-2 for read-only slaves, higher for
-         masters w/o cache warming.
-      -->
-    <maxWarmingSearchers>2</maxWarmingSearchers>
-
   </query>
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml b/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml
index ea50610..4b0899b 100644
--- a/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml
+++ b/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml
@@ -599,17 +599,6 @@
       -->
     <useColdSearcher>false</useColdSearcher>
 
-    <!-- Max Warming Searchers
-         
-         Maximum number of searchers that may be warming in the
-         background concurrently.  An error is returned if this limit
-         is exceeded.
-
-         Recommend values of 1-2 for read-only slaves, higher for
-         masters w/o cache warming.
-      -->
-    <maxWarmingSearchers>2</maxWarmingSearchers>
-
   </query>
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml b/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml
index 65b348a..e4b0526 100644
--- a/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml
+++ b/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml
@@ -611,17 +611,6 @@
       -->
     <useColdSearcher>false</useColdSearcher>
 
-    <!-- Max Warming Searchers
-         
-         Maximum number of searchers that may be warming in the
-         background concurrently.  An error is returned if this limit
-         is exceeded.
-
-         Recommend values of 1-2 for read-only slaves, higher for
-         masters w/o cache warming.
-      -->
-    <maxWarmingSearchers>2</maxWarmingSearchers>
-
   </query>
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/solrj/src/test/org/apache/solr/client/solrj/LargeVolumeTestBase.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/LargeVolumeTestBase.java b/solr/solrj/src/test/org/apache/solr/client/solrj/LargeVolumeTestBase.java
index b9d525c..8f43c33 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/LargeVolumeTestBase.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/LargeVolumeTestBase.java
@@ -60,7 +60,8 @@ public abstract class LargeVolumeTestBase extends SolrJettyTestBase
 
     // some of the commits could have failed because maxWarmingSearchers exceeded,
     // so do a final commit to make sure everything is visible.
-    client.commit();
+    // This should no longer be true as of SOLR-9712 (Solr 6.4)
+    // client.commit();
     
     query(threadCount * numdocs);
     log.info("done");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTestBase.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTestBase.java b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTestBase.java
index ce762a0..114d8ed 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTestBase.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTestBase.java
@@ -38,7 +38,6 @@ abstract public class SolrExampleTestBase extends AbstractSolrTestCase {
   
   @Override
   public void setUp() throws Exception {
-    ignoreException("maxWarmingSearchers");
     super.setUp();
     
     // this sets the property for jetty starting SolrDispatchFilter

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/test-framework/src/java/org/apache/solr/SolrJettyTestBase.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/SolrJettyTestBase.java b/solr/test-framework/src/java/org/apache/solr/SolrJettyTestBase.java
index 0badec1..a936ee5 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrJettyTestBase.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrJettyTestBase.java
@@ -110,8 +110,6 @@ abstract public class SolrJettyTestBase extends SolrTestCaseJ4
     nodeProps.setProperty("coreRootDirectory", coresDir.toString());
     nodeProps.setProperty("configSetBaseDir", solrHome);
 
-    ignoreException("maxWarmingSearchers");
-
     jetty = new JettySolrRunner(solrHome, nodeProps, jettyConfig);
     jetty.start();
     port = jetty.getLocalPort();


[33/44] lucene-solr:jira/solr-9854: SOLR-9870: fix two typos in SolrCore.java (Mike Drob via Christine Poerschke)

Posted by ab...@apache.org.
SOLR-9870: fix two typos in SolrCore.java (Mike Drob via Christine Poerschke)


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

Branch: refs/heads/jira/solr-9854
Commit: 07609443c5c7675c41e8e5ac3b62576f41ee4be8
Parents: b22451f
Author: Christine Poerschke <cp...@apache.org>
Authored: Mon Dec 19 16:05:06 2016 +0000
Committer: Christine Poerschke <cp...@apache.org>
Committed: Mon Dec 19 16:14:51 2016 +0000

----------------------------------------------------------------------
 solr/core/src/java/org/apache/solr/core/SolrCore.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/07609443/solr/core/src/java/org/apache/solr/core/SolrCore.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrCore.java b/solr/core/src/java/org/apache/solr/core/SolrCore.java
index b3facef..f75e780 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -1701,7 +1701,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
    *
    * realtimeSearcher is updated to the latest opened searcher, regardless of the value of "realtime".
    *
-   * This method acquires openSearcherLock - do not call with searckLock held!
+   * This method acquires openSearcherLock - do not call with searchLock held!
    */
   public RefCounted<SolrIndexSearcher>  openNewSearcher(boolean updateHandlerReopens, boolean realtime) {
     if (isClosed()) { // catch some errors quicker
@@ -1982,7 +1982,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
 
       Future future=null;
 
-      // if the underlying seracher has not changed, no warming is needed
+      // if the underlying searcher has not changed, no warming is needed
       if (newSearcher != currSearcher) {
 
         // warm the new searcher based on the current searcher.


[13/44] lucene-solr:jira/solr-9854: LUCENE-7589: Prevent outliers from raising the bpv for everyone.

Posted by ab...@apache.org.
LUCENE-7589: Prevent outliers from raising the bpv for everyone.


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

Branch: refs/heads/jira/solr-9854
Commit: 3b182aa2fb3e4062f6ec5be819f3aa70aa2e523d
Parents: ea1569e
Author: Adrien Grand <jp...@gmail.com>
Authored: Thu Dec 15 16:33:36 2016 +0100
Committer: Adrien Grand <jp...@gmail.com>
Committed: Thu Dec 15 17:17:54 2016 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   4 +
 .../lucene70/Lucene70DocValuesConsumer.java     | 163 +++++++++++---
 .../lucene70/Lucene70DocValuesFormat.java       |   5 +-
 .../lucene70/Lucene70DocValuesProducer.java     | 220 +++++++++++++++----
 .../java/org/apache/lucene/util/LongValues.java |   9 +
 .../apache/lucene/util/packed/DirectWriter.java |   8 +-
 .../lucene70/TestLucene70DocValuesFormat.java   | 152 +++++++++++++
 7 files changed, 479 insertions(+), 82 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b182aa2/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index bacc270..7e61469 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -48,6 +48,10 @@ Optimizations
 * LUCENE-7519: Add optimized APIs to compute browse-only top level
   facets (Mike McCandless)
 
+* LUCENE-7589: Numeric doc values now have the ability to encode blocks of
+  values using different numbers of bits per value if this proves to save
+  storage. (Adrien Grand)
+
 Other
 
 * LUCENE-7328: Remove LegacyNumericEncoding from GeoPointField. (Nick Knize)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b182aa2/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesConsumer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesConsumer.java
index e1b66e1..2dd68e9 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesConsumer.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesConsumer.java
@@ -18,6 +18,8 @@ package org.apache.lucene.codecs.lucene70;
 
 
 import static org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
+import static org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat.NUMERIC_BLOCK_SHIFT;
+import static org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE;
 
 import java.io.Closeable; // javadocs
 import java.io.IOException;
@@ -42,6 +44,7 @@ import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.SortedSetSelector;
+import org.apache.lucene.store.GrowableByteArrayDataOutput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.BytesRef;
@@ -112,12 +115,46 @@ final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Close
     });
   }
 
+  private static class MinMaxTracker {
+    long min, max, numValues, spaceInBits;
+
+    MinMaxTracker() {
+      reset();
+      spaceInBits = 0;
+    }
+
+    private void reset() {
+      min = Long.MAX_VALUE;
+      max = Long.MIN_VALUE;
+      numValues = 0;
+    }
+
+    /** Accumulate a new value. */
+    void update(long v) {
+      min = Math.min(min, v);
+      max = Math.max(max, v);
+      ++numValues;
+    }
+
+    /** Update the required space. */
+    void finish() {
+      if (max > min) {
+        spaceInBits += DirectWriter.unsignedBitsRequired(max - min) * numValues;
+      }
+    }
+
+    /** Update space usage and get ready for accumulating values for the next block. */
+    void nextBlock() {
+      finish();
+      reset();
+    }
+  }
+
   private long[] writeValues(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
     SortedNumericDocValues values = valuesProducer.getSortedNumeric(field);
     int numDocsWithValue = 0;
-    long numValues = 0;
-    long min = Long.MAX_VALUE;
-    long max = Long.MIN_VALUE;
+    MinMaxTracker minMax = new MinMaxTracker();
+    MinMaxTracker blockMinMax = new MinMaxTracker();
     long gcd = 0;
     Set<Long> uniqueValues = new HashSet<>();
     for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
@@ -130,26 +167,35 @@ final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Close
             // wrong results. Since these extreme values are unlikely, we just discard
             // GCD computation for them
             gcd = 1;
-          } else if (numValues != 0) { // minValue needs to be set first
-            gcd = MathUtil.gcd(gcd, v - min);
+          } else if (minMax.numValues != 0) { // minValue needs to be set first
+            gcd = MathUtil.gcd(gcd, v - minMax.min);
           }
         }
 
-        min = Math.min(min, v);
-        max = Math.max(max, v);
+        minMax.update(v);
+        blockMinMax.update(v);
+        if (blockMinMax.numValues == NUMERIC_BLOCK_SIZE) {
+          blockMinMax.nextBlock();
+        }
 
         if (uniqueValues != null
             && uniqueValues.add(v)
             && uniqueValues.size() > 256) {
           uniqueValues = null;
         }
-
-        numValues++;
       }
 
       numDocsWithValue++;
     }
 
+    minMax.finish();
+    blockMinMax.finish();
+
+    final long numValues = minMax.numValues;
+    long min = minMax.min;
+    final long max = minMax.max;
+    assert blockMinMax.spaceInBits <= minMax.spaceInBits;
+
     if (numDocsWithValue == 0) {
       meta.writeLong(-2);
       meta.writeLong(0L);
@@ -166,6 +212,7 @@ final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Close
 
     meta.writeLong(numValues);
     final int numBitsPerValue;
+    boolean doBlocks = false;
     Map<Long, Integer> encode = null;
     if (min >= max) {
       numBitsPerValue = 0;
@@ -189,12 +236,19 @@ final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Close
         gcd = 1;
       } else {
         uniqueValues = null;
-        numBitsPerValue = DirectWriter.unsignedBitsRequired((max - min) / gcd);
-        if (gcd == 1 && min > 0
-            && DirectWriter.unsignedBitsRequired(max) == DirectWriter.unsignedBitsRequired(max - min)) {
-          min = 0;
+        // we do blocks if that appears to save 10+% storage
+        doBlocks = minMax.spaceInBits > 0 && (double) blockMinMax.spaceInBits / minMax.spaceInBits <= 0.9;
+        if (doBlocks) {
+          numBitsPerValue = 0xFF;
+          meta.writeInt(-2 - NUMERIC_BLOCK_SHIFT);
+        } else {
+          numBitsPerValue = DirectWriter.unsignedBitsRequired((max - min) / gcd);
+          if (gcd == 1 && min > 0
+              && DirectWriter.unsignedBitsRequired(max) == DirectWriter.unsignedBitsRequired(max - min)) {
+            min = 0;
+          }
+          meta.writeInt(-1);
         }
-        meta.writeInt(-1);
       }
     }
 
@@ -203,26 +257,79 @@ final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Close
     meta.writeLong(gcd);
     long startOffset = data.getFilePointer();
     meta.writeLong(startOffset);
-    if (numBitsPerValue != 0) {
-      values = valuesProducer.getSortedNumeric(field);
-      DirectWriter writer = DirectWriter.getInstance(data, numValues, numBitsPerValue);
-      for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
-        for (int i = 0, count = values.docValueCount(); i < count; ++i) {
-          long v = values.nextValue();
-          if (encode == null) {
-            writer.add((v - min) / gcd);
-          } else {
-            writer.add(encode.get(v));
-          }
-        }
-      }
-      writer.finish();
+    if (doBlocks) {
+      writeValuesMultipleBlocks(valuesProducer.getSortedNumeric(field), gcd);
+    } else if (numBitsPerValue != 0) {
+      writeValuesSingleBlock(valuesProducer.getSortedNumeric(field), numValues, numBitsPerValue, min, gcd, encode);
     }
     meta.writeLong(data.getFilePointer() - startOffset);
 
     return new long[] {numDocsWithValue, numValues};
   }
 
+  private void writeValuesSingleBlock(SortedNumericDocValues values, long numValues, int numBitsPerValue,
+      long min, long gcd, Map<Long, Integer> encode) throws IOException {
+    DirectWriter writer = DirectWriter.getInstance(data, numValues, numBitsPerValue);
+    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
+      for (int i = 0, count = values.docValueCount(); i < count; ++i) {
+        long v = values.nextValue();
+        if (encode == null) {
+          writer.add((v - min) / gcd);
+        } else {
+          writer.add(encode.get(v));
+        }
+      }
+    }
+    writer.finish();
+  }
+ 
+  private void writeValuesMultipleBlocks(SortedNumericDocValues values, long gcd) throws IOException {
+    final long[] buffer = new long[NUMERIC_BLOCK_SIZE];
+    final GrowableByteArrayDataOutput encodeBuffer = new GrowableByteArrayDataOutput(NUMERIC_BLOCK_SIZE);
+    int upTo = 0;
+    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
+      for (int i = 0, count = values.docValueCount(); i < count; ++i) {
+        buffer[upTo++] = values.nextValue();
+        if (upTo == NUMERIC_BLOCK_SIZE) {
+          writeBlock(buffer, NUMERIC_BLOCK_SIZE, gcd, encodeBuffer);
+          upTo = 0;
+        }
+      }
+    }
+    if (upTo > 0) {
+      writeBlock(buffer, upTo, gcd, encodeBuffer);
+    }
+  }
+
+  private void writeBlock(long[] values, int length, long gcd, GrowableByteArrayDataOutput buffer) throws IOException {
+    assert length > 0;
+    long min = values[0];
+    long max = values[0];
+    for (int i = 1; i < length; ++i) {
+      final long v = values[i];
+      assert Math.floorMod(values[i] - min, gcd) == 0;
+      min = Math.min(min, v);
+      max = Math.max(max, v);
+    }
+    if (min == max) {
+      data.writeByte((byte) 0);
+      data.writeLong(min);
+    } else {
+      final int bitsPerValue = DirectWriter.unsignedBitsRequired(max - min);
+      buffer.reset();
+      assert buffer.getPosition() == 0;
+      final DirectWriter w = DirectWriter.getInstance(buffer, length, bitsPerValue);
+      for (int i = 0; i < length; ++i) {
+        w.add((values[i] - min) / gcd);
+      }
+      w.finish();
+      data.writeByte((byte) bitsPerValue);
+      data.writeLong(min);
+      data.writeInt(buffer.getPosition());
+      data.writeBytes(buffer.getBytes(), buffer.getPosition());
+    }
+  }
+
   @Override
   public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
     meta.writeInt(field.number);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b182aa2/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesFormat.java
index ee477d6..2ce2124 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesFormat.java
@@ -146,10 +146,11 @@ public final class Lucene70DocValuesFormat extends DocValuesFormat {
   static final byte SORTED_SET = 3;
   static final byte SORTED_NUMERIC = 4;
 
-  // addressing uses 16k blocks
-  static final int MONOTONIC_BLOCK_SIZE = 16384;
   static final int DIRECT_MONOTONIC_BLOCK_SHIFT = 16;
 
+  static final int NUMERIC_BLOCK_SHIFT = 14;
+  static final int NUMERIC_BLOCK_SIZE = 1 << NUMERIC_BLOCK_SHIFT;
+
   static final int TERMS_DICT_BLOCK_SHIFT = 4;
   static final int TERMS_DICT_BLOCK_SIZE = 1 << TERMS_DICT_BLOCK_SHIFT;
   static final int TERMS_DICT_BLOCK_MASK = TERMS_DICT_BLOCK_SIZE - 1;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b182aa2/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesProducer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesProducer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesProducer.java
index 3f3e73f..386655e 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesProducer.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesProducer.java
@@ -144,7 +144,7 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
     entry.docsWithFieldLength = meta.readLong();
     entry.numValues = meta.readLong();
     int tableSize = meta.readInt();
-    if (tableSize < -1 || tableSize > 256) {
+    if (tableSize > 256) {
       throw new CorruptIndexException("invalid table size: " + tableSize, meta);
     }
     if (tableSize >= 0) {
@@ -154,6 +154,11 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
         entry.table[i] = meta.readLong();
       }
     }
+    if (tableSize < -1) {
+      entry.blockShift = -2 - tableSize;
+    } else {
+      entry.blockShift = -1;
+    }
     entry.bitsPerValue = meta.readByte();
     entry.minValue = meta.readLong();
     entry.gcd = meta.readLong();
@@ -260,6 +265,7 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
 
   private static class NumericEntry {
     long[] table;
+    int blockShift;
     byte bitsPerValue;
     long docsWithFieldOffset;
     long docsWithFieldLength;
@@ -429,24 +435,62 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
         };
       } else {
         final RandomAccessInput slice = data.randomAccessSlice(entry.valuesOffset, entry.valuesLength);
-        final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue);
-        if (entry.table != null) {
-          final long[] table = entry.table;
-          return new DenseNumericDocValues(maxDoc) {
-            @Override
-            public long longValue() throws IOException {
-              return table[(int) values.get(doc)];
-            }
-          };
-        } else {
+        if (entry.blockShift >= 0) {
+          // dense but split into blocks of different bits per value
+          final int shift = entry.blockShift;
           final long mul = entry.gcd;
-          final long delta = entry.minValue;
+          final int mask = (1 << shift) - 1;
           return new DenseNumericDocValues(maxDoc) {
+            int block = -1;
+            long delta;
+            long offset;
+            long blockEndOffset;
+            LongValues values;
+
             @Override
             public long longValue() throws IOException {
-              return mul * values.get(doc) + delta;
+              final int block = doc >>> shift;
+              if (this.block != block) {
+                int bitsPerValue;
+                do {
+                  offset = blockEndOffset;
+                  bitsPerValue = slice.readByte(offset++);
+                  delta = slice.readLong(offset);
+                  offset += Long.BYTES;
+                  if (bitsPerValue == 0) {
+                    blockEndOffset = offset;
+                  } else {
+                    final int length = slice.readInt(offset);
+                    offset += Integer.BYTES;
+                    blockEndOffset = offset + length;
+                  }
+                  this.block ++;
+                } while (this.block != block);
+                values = bitsPerValue == 0 ? LongValues.ZEROES : DirectReader.getInstance(slice, bitsPerValue, offset);
+              }
+              return mul * values.get(doc & mask) + delta;
             }
           };
+        } else {
+          final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue);
+          if (entry.table != null) {
+            final long[] table = entry.table;
+            return new DenseNumericDocValues(maxDoc) {
+              @Override
+              public long longValue() throws IOException {
+                return table[(int) values.get(doc)];
+              }
+            };
+          } else {
+            final long mul = entry.gcd;
+            final long delta = entry.minValue;
+            return new DenseNumericDocValues(maxDoc) {
+              @Override
+              public long longValue() throws IOException {
+                return mul * values.get(doc) + delta;
+              }
+            };
+          }
         }
       }
     } else {
@@ -461,24 +505,63 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
         };
       } else {
         final RandomAccessInput slice = data.randomAccessSlice(entry.valuesOffset, entry.valuesLength);
-        final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue);
-        if (entry.table != null) {
-          final long[] table = entry.table;
-          return new SparseNumericDocValues(disi) {
-            @Override
-            public long longValue() throws IOException {
-              return table[(int) values.get(disi.index())];
-            }
-          };
-        } else {
+        if (entry.blockShift >= 0) {
+          // sparse and split into blocks of different bits per value
+          final int shift = entry.blockShift;
           final long mul = entry.gcd;
-          final long delta = entry.minValue;
+          final int mask = (1 << shift) - 1;
           return new SparseNumericDocValues(disi) {
+            int block = -1;
+            long delta;
+            long offset;
+            long blockEndOffset;
+            LongValues values;
+
             @Override
             public long longValue() throws IOException {
-              return mul * values.get(disi.index()) + delta;
+              final int index = disi.index();
+              final int block = index >>> shift;
+              if (this.block != block) {
+                int bitsPerValue;
+                do {
+                  offset = blockEndOffset;
+                  bitsPerValue = slice.readByte(offset++);
+                  delta = slice.readLong(offset);
+                  offset += Long.BYTES;
+                  if (bitsPerValue == 0) {
+                    blockEndOffset = offset;
+                  } else {
+                    final int length = slice.readInt(offset);
+                    offset += Integer.BYTES;
+                    blockEndOffset = offset + length;
+                  }
+                  this.block ++;
+                } while (this.block != block);
+                values = bitsPerValue == 0 ? LongValues.ZEROES : DirectReader.getInstance(slice, bitsPerValue, offset);
+              }
+              return mul * values.get(index & mask) + delta;
             }
           };
+        } else {
+          final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue);
+          if (entry.table != null) {
+            final long[] table = entry.table;
+            return new SparseNumericDocValues(disi) {
+              @Override
+              public long longValue() throws IOException {
+                return table[(int) values.get(disi.index())];
+              }
+            };
+          } else {
+            final long mul = entry.gcd;
+            final long delta = entry.minValue;
+            return new SparseNumericDocValues(disi) {
+              @Override
+              public long longValue() throws IOException {
+                return mul * values.get(disi.index()) + delta;
+              }
+            };
+          }
         }
       }
     }
@@ -494,34 +577,75 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
       };
     } else {
       final RandomAccessInput slice = data.randomAccessSlice(entry.valuesOffset, entry.valuesLength);
-      final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue);
-      if (entry.table != null) {
-        final long[] table = entry.table;
+      if (entry.blockShift >= 0) {
+        final int shift = entry.blockShift;
+        final long mul = entry.gcd;
+        final long mask = (1L << shift) - 1;
         return new LongValues() {
-          @Override
-          public long get(long index) {
-            return table[(int) values.get(index)];
-          }
-        };
-      } else if (entry.gcd != 1) {
-        final long gcd = entry.gcd;
-        final long minValue = entry.minValue;
-        return new LongValues() {
-          @Override
-          public long get(long index) {
-            return values.get(index) * gcd + minValue;
-          }
-        };
-      } else if (entry.minValue != 0) {
-        final long minValue = entry.minValue;
-        return new LongValues() {
-          @Override
+          long block = -1;
+          long delta;
+          long offset;
+          long blockEndOffset;
+          LongValues values;
+
           public long get(long index) {
-            return values.get(index) + minValue;
+            final long block = index >>> shift;
+            if (this.block != block) {
+              assert block > this.block : "Reading backwards is illegal: " + this.block + " < " + block;
+              int bitsPerValue;
+              do {
+                offset = blockEndOffset;
+                try {
+                  bitsPerValue = slice.readByte(offset++);
+                  delta = slice.readLong(offset);
+                  offset += Long.BYTES;
+                  if (bitsPerValue == 0) {
+                    blockEndOffset = offset;
+                  } else {
+                    final int length = slice.readInt(offset);
+                    offset += Integer.BYTES;
+                    blockEndOffset = offset + length;
+                  }
+                } catch (IOException e) {
+                  throw new RuntimeException(e);
+                }
+                this.block ++;
+              } while (this.block != block);
+              values = bitsPerValue == 0 ? LongValues.ZEROES : DirectReader.getInstance(slice, bitsPerValue, offset);
+            }
+            return mul * values.get(index & mask) + delta;
           }
         };
       } else {
-        return values;
+        final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue);
+        if (entry.table != null) {
+          final long[] table = entry.table;
+          return new LongValues() {
+            @Override
+            public long get(long index) {
+              return table[(int) values.get(index)];
+            }
+          };
+        } else if (entry.gcd != 1) {
+          final long gcd = entry.gcd;
+          final long minValue = entry.minValue;
+          return new LongValues() {
+            @Override
+            public long get(long index) {
+              return values.get(index) * gcd + minValue;
+            }
+          };
+        } else if (entry.minValue != 0) {
+          final long minValue = entry.minValue;
+          return new LongValues() {
+            @Override
+            public long get(long index) {
+              return values.get(index) + minValue;
+            }
+          };
+        } else {
+          return values;
+        }
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b182aa2/lucene/core/src/java/org/apache/lucene/util/LongValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/LongValues.java b/lucene/core/src/java/org/apache/lucene/util/LongValues.java
index 23f4d32..04fbf81 100644
--- a/lucene/core/src/java/org/apache/lucene/util/LongValues.java
+++ b/lucene/core/src/java/org/apache/lucene/util/LongValues.java
@@ -30,6 +30,15 @@ public abstract class LongValues  {
 
   };
 
+  public static final LongValues ZEROES = new LongValues() {
+
+    @Override
+    public long get(long index) {
+      return 0;
+    }
+
+  };
+
   /** Get value at <code>index</code>. */
   public abstract long get(long index);
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b182aa2/lucene/core/src/java/org/apache/lucene/util/packed/DirectWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/packed/DirectWriter.java b/lucene/core/src/java/org/apache/lucene/util/packed/DirectWriter.java
index 9a7f18e..5a38445 100644
--- a/lucene/core/src/java/org/apache/lucene/util/packed/DirectWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/util/packed/DirectWriter.java
@@ -21,7 +21,7 @@ import java.io.EOFException;
 import java.io.IOException;
 import java.util.Arrays;
 
-import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.DataOutput;
 
 /** 
  * Class for writing packed integers to be directly read from Directory.
@@ -44,7 +44,7 @@ import org.apache.lucene.store.IndexOutput;
 public final class DirectWriter {
   final int bitsPerValue;
   final long numValues;
-  final IndexOutput output;
+  final DataOutput output;
   
   long count;
   boolean finished;
@@ -56,7 +56,7 @@ public final class DirectWriter {
   final BulkOperation encoder;
   final int iterations;
   
-  DirectWriter(IndexOutput output, long numValues, int bitsPerValue) {
+  DirectWriter(DataOutput output, long numValues, int bitsPerValue) {
     this.output = output;
     this.numValues = numValues;
     this.bitsPerValue = bitsPerValue;
@@ -103,7 +103,7 @@ public final class DirectWriter {
   }
   
   /** Returns an instance suitable for encoding {@code numValues} using {@code bitsPerValue} */
-  public static DirectWriter getInstance(IndexOutput output, long numValues, int bitsPerValue) {
+  public static DirectWriter getInstance(DataOutput output, long numValues, int bitsPerValue) {
     if (Arrays.binarySearch(SUPPORTED_BITS_PER_VALUE, bitsPerValue) < 0) {
       throw new IllegalArgumentException("Unsupported bitsPerValue " + bitsPerValue + ". Did you use bitsRequired?");
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b182aa2/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70DocValuesFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70DocValuesFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70DocValuesFormat.java
index 8661298..6cca55e 100644
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70DocValuesFormat.java
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70DocValuesFormat.java
@@ -25,6 +25,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 import java.util.TreeSet;
+import java.util.function.LongSupplier;
 import java.util.function.Supplier;
 
 import org.apache.lucene.analysis.MockAnalyzer;
@@ -61,6 +62,7 @@ import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum.SeekStatus;
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.RAMFile;
@@ -534,4 +536,154 @@ public class TestLucene70DocValuesFormat extends BaseCompressingDocValuesFormatT
       dir.close();
     }
   }
+
+  @Slow
+  public void testSortedNumericBlocksOfVariousBitsPerValue() throws Exception {
+    doTestSortedNumericBlocksOfVariousBitsPerValue(() -> TestUtil.nextInt(random(), 1, 3));
+  }
+
+  @Slow
+  public void testSparseSortedNumericBlocksOfVariousBitsPerValue() throws Exception {
+    doTestSortedNumericBlocksOfVariousBitsPerValue(() -> TestUtil.nextInt(random(), 0, 2));
+  }
+
+  @Slow
+  public void testNumericBlocksOfVariousBitsPerValue() throws Exception {
+    doTestSparseNumericBlocksOfVariousBitsPerValue(1);
+  }
+
+  @Slow
+  public void testSparseNumericBlocksOfVariousBitsPerValue() throws Exception {
+    doTestSparseNumericBlocksOfVariousBitsPerValue(random().nextDouble());
+  }
+
+  private static LongSupplier blocksOfVariousBPV() {
+    final long mul = TestUtil.nextInt(random(), 1, 100);
+    final long min = random().nextInt();
+    return new LongSupplier() {
+      int i = Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE;
+      int maxDelta;
+      @Override
+      public long getAsLong() {
+        if (i == Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE) {
+          maxDelta = 1 << random().nextInt(5);
+          i = 0;
+        }
+        i++;
+        return min + mul * random().nextInt(maxDelta);
+      }
+    };
+  }
+
+  private void doTestSortedNumericBlocksOfVariousBitsPerValue(LongSupplier counts) throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
+    conf.setMaxBufferedDocs(atLeast(Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE));
+    conf.setRAMBufferSizeMB(-1);
+    conf.setMergePolicy(newLogMergePolicy(random().nextBoolean()));
+    IndexWriter writer = new IndexWriter(dir, conf);
+    
+    final int numDocs = atLeast(Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE*3);
+    final LongSupplier values = blocksOfVariousBPV();
+    for (int i = 0; i < numDocs; i++) {
+      Document doc = new Document();
+      
+      int valueCount = (int) counts.getAsLong();
+      long valueArray[] = new long[valueCount];
+      for (int j = 0; j < valueCount; j++) {
+        long value = values.getAsLong();
+        valueArray[j] = value;
+        doc.add(new SortedNumericDocValuesField("dv", value));
+      }
+      Arrays.sort(valueArray);
+      for (int j = 0; j < valueCount; j++) {
+        doc.add(new StoredField("stored", Long.toString(valueArray[j])));
+      }
+      writer.addDocument(doc);
+      if (random().nextInt(31) == 0) {
+        writer.commit();
+      }
+    }
+    writer.forceMerge(1);
+
+    writer.close();
+    
+    // compare
+    DirectoryReader ir = DirectoryReader.open(dir);
+    TestUtil.checkReader(ir);
+    for (LeafReaderContext context : ir.leaves()) {
+      LeafReader r = context.reader();
+      SortedNumericDocValues docValues = DocValues.getSortedNumeric(r, "dv");
+      for (int i = 0; i < r.maxDoc(); i++) {
+        if (i > docValues.docID()) {
+          docValues.nextDoc();
+        }
+        String expected[] = r.document(i).getValues("stored");
+        if (i < docValues.docID()) {
+          assertEquals(0, expected.length);
+        } else {
+          String actual[] = new String[docValues.docValueCount()];
+          for (int j = 0; j < actual.length; j++) {
+            actual[j] = Long.toString(docValues.nextValue());
+          }
+          assertArrayEquals(expected, actual);
+        }
+      }
+    }
+    ir.close();
+    dir.close();
+  }
+
+  private void doTestSparseNumericBlocksOfVariousBitsPerValue(double density) throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
+    conf.setMaxBufferedDocs(atLeast(Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE));
+    conf.setRAMBufferSizeMB(-1);
+    conf.setMergePolicy(newLogMergePolicy(random().nextBoolean()));
+    IndexWriter writer = new IndexWriter(dir, conf);
+    Document doc = new Document();
+    Field storedField = newStringField("stored", "", Field.Store.YES);
+    Field dvField = new NumericDocValuesField("dv", 0);
+    doc.add(storedField);
+    doc.add(dvField);
+
+    final int numDocs = atLeast(Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE*3);
+    final LongSupplier longs = blocksOfVariousBPV();
+    for (int i = 0; i < numDocs; i++) {
+      if (random().nextDouble() > density) {
+        writer.addDocument(new Document());
+        continue;
+      }
+      long value = longs.getAsLong();
+      storedField.setStringValue(Long.toString(value));
+      dvField.setLongValue(value);
+      writer.addDocument(doc);
+    }
+
+    writer.forceMerge(1);
+
+    writer.close();
+    
+    // compare
+    DirectoryReader ir = DirectoryReader.open(dir);
+    TestUtil.checkReader(ir);
+    for (LeafReaderContext context : ir.leaves()) {
+      LeafReader r = context.reader();
+      NumericDocValues docValues = DocValues.getNumeric(r, "dv");
+      docValues.nextDoc();
+      for (int i = 0; i < r.maxDoc(); i++) {
+        String storedValue = r.document(i).get("stored");
+        if (storedValue == null) {
+          assertTrue(docValues.docID() > i);
+        } else {
+          assertEquals(i, docValues.docID());
+          assertEquals(Long.parseLong(storedValue), docValues.longValue());
+          docValues.nextDoc();
+        }
+      }
+      assertEquals(DocIdSetIterator.NO_MORE_DOCS, docValues.docID());
+    }
+    ir.close();
+    dir.close();
+  }
 }


[03/44] lucene-solr:jira/solr-9854: Fix LeafReader.getNumericDocValues javadoc

Posted by ab...@apache.org.
Fix LeafReader.getNumericDocValues javadoc


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

Branch: refs/heads/jira/solr-9854
Commit: 770f1eb8ad6af5cce55d1bdf52f1288216c9691f
Parents: ad7152a
Author: Shai Erera <sh...@apache.org>
Authored: Wed Dec 14 13:07:19 2016 +0200
Committer: Shai Erera <sh...@apache.org>
Committed: Wed Dec 14 13:07:19 2016 +0200

----------------------------------------------------------------------
 lucene/core/src/java/org/apache/lucene/index/LeafReader.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/770f1eb8/lucene/core/src/java/org/apache/lucene/index/LeafReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/LeafReader.java b/lucene/core/src/java/org/apache/lucene/index/LeafReader.java
index acdd0d8..73394f2 100644
--- a/lucene/core/src/java/org/apache/lucene/index/LeafReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/LeafReader.java
@@ -242,7 +242,7 @@ public abstract class LeafReader extends IndexReader {
   /** Returns {@link NumericDocValues} for this field, or
    *  null if no numeric doc values were indexed for
    *  this field.  The returned instance should only be
-   *  used by a single thread.  This will never return null. */
+   *  used by a single thread. */
   public abstract NumericDocValues getNumericDocValues(String field) throws IOException;
 
   /** Returns {@link BinaryDocValues} for this field, or


[34/44] lucene-solr:jira/solr-9854: SOLR-8542: techproducts example now includes (disabled) learning-to-rank support (enable via -Dsolr.ltr.enabled=true)

Posted by ab...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c8542b2b/solr/contrib/ltr/src/java/org/apache/solr/ltr/model/LinearModel.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/java/org/apache/solr/ltr/model/LinearModel.java b/solr/contrib/ltr/src/java/org/apache/solr/ltr/model/LinearModel.java
index 57fc5ad..ee065c3 100644
--- a/solr/contrib/ltr/src/java/org/apache/solr/ltr/model/LinearModel.java
+++ b/solr/contrib/ltr/src/java/org/apache/solr/ltr/model/LinearModel.java
@@ -47,6 +47,16 @@ import org.apache.solr.ltr.norm.Normalizer;
    }
 }</pre>
  * <p>
+ * Training libraries:
+ * <ul>
+ * <li> <a href="https://www.csie.ntu.edu.tw/~cjlin/liblinear/">
+ * LIBLINEAR -- A Library for Large Linear Classification</a>
+ * </ul>
+ * <ul>
+ * <li> <a href="https://www.csie.ntu.edu.tw/~cjlin/libsvm/">
+ * LIBSVM -- A Library for Support Vector Machines</a>
+ * </ul>
+ * <p>
  * Background reading:
  * <ul>
  * <li> <a href="http://www.cs.cornell.edu/people/tj/publications/joachims_02c.pdf">

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c8542b2b/solr/contrib/ltr/src/java/org/apache/solr/ltr/model/MultipleAdditiveTreesModel.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/java/org/apache/solr/ltr/model/MultipleAdditiveTreesModel.java b/solr/contrib/ltr/src/java/org/apache/solr/ltr/model/MultipleAdditiveTreesModel.java
index 4fa595e..926fab4 100644
--- a/solr/contrib/ltr/src/java/org/apache/solr/ltr/model/MultipleAdditiveTreesModel.java
+++ b/solr/contrib/ltr/src/java/org/apache/solr/ltr/model/MultipleAdditiveTreesModel.java
@@ -71,6 +71,11 @@ import org.apache.solr.util.SolrPluginUtils;
    }
 }</pre>
  * <p>
+ * Training libraries:
+ * <ul>
+ * <li> <a href="http://sourceforge.net/p/lemur/wiki/RankLib/">RankLib</a>
+ * </ul>
+ * <p>
  * Background reading:
  * <ul>
  * <li> <a href="http://research.microsoft.com/pubs/132652/MSR-TR-2010-82.pdf">

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c8542b2b/solr/contrib/ltr/src/java/org/apache/solr/ltr/response/transform/LTRFeatureLoggerTransformerFactory.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/java/org/apache/solr/ltr/response/transform/LTRFeatureLoggerTransformerFactory.java b/solr/contrib/ltr/src/java/org/apache/solr/ltr/response/transform/LTRFeatureLoggerTransformerFactory.java
index 93ebe63..354ecc2 100644
--- a/solr/contrib/ltr/src/java/org/apache/solr/ltr/response/transform/LTRFeatureLoggerTransformerFactory.java
+++ b/solr/contrib/ltr/src/java/org/apache/solr/ltr/response/transform/LTRFeatureLoggerTransformerFactory.java
@@ -17,6 +17,7 @@
 package org.apache.solr.ltr.response.transform;
 
 import java.io.IOException;
+import java.lang.invoke.MethodHandles;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
@@ -27,6 +28,7 @@ import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.ltr.CSVFeatureLogger;
 import org.apache.solr.ltr.FeatureLogger;
 import org.apache.solr.ltr.LTRRescorer;
 import org.apache.solr.ltr.LTRScoringQuery;
@@ -44,17 +46,25 @@ import org.apache.solr.response.transform.DocTransformer;
 import org.apache.solr.response.transform.TransformerFactory;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.util.SolrPluginUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This transformer will take care to generate and append in the response the
- * features declared in the feature store of the current model. The class is
- * useful if you are not interested in the reranking (e.g., bootstrapping a
- * machine learning framework).
- */
+ * features declared in the feature store of the current reranking model,
+ * or a specified feature store.  Ex. <code>fl=id,[features store=myStore efi.user_text="ibm"]</code>
+ * 
+ * <h3>Parameters</h3>
+ * <code>store</code> - The feature store to extract features from. If not provided it
+ * will default to the features used by your reranking model.<br>
+ * <code>efi.*</code> - External feature information variables required by the features
+ * you are extracting.<br>
+ * <code>format</code> - The format you want the features to be returned in.  Supports (dense|sparse). Defaults to sparse.<br>
+*/
+
 public class LTRFeatureLoggerTransformerFactory extends TransformerFactory {
 
-  // used inside fl to specify the output format (csv/json) of the extracted features
-  private static final String FV_RESPONSE_WRITER = "fvwt";
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   // used inside fl to specify the format (dense|sparse) of the extracted features
   private static final String FV_FORMAT = "format";
@@ -64,27 +74,43 @@ public class LTRFeatureLoggerTransformerFactory extends TransformerFactory {
 
   private static String DEFAULT_LOGGING_MODEL_NAME = "logging-model";
 
+  private String fvCacheName;
   private String loggingModelName = DEFAULT_LOGGING_MODEL_NAME;
-  private String defaultFvStore;
-  private String defaultFvwt;
-  private String defaultFvFormat;
+  private String defaultStore;
+  private String defaultFormat;
+  private char csvKeyValueDelimiter = CSVFeatureLogger.DEFAULT_KEY_VALUE_SEPARATOR;
+  private char csvFeatureSeparator = CSVFeatureLogger.DEFAULT_FEATURE_SEPARATOR;
 
   private LTRThreadModule threadManager = null;
 
+  public void setFvCacheName(String fvCacheName) {
+    this.fvCacheName = fvCacheName;
+  }
+
   public void setLoggingModelName(String loggingModelName) {
     this.loggingModelName = loggingModelName;
   }
 
-  public void setStore(String defaultFvStore) {
-    this.defaultFvStore = defaultFvStore;
+  public void setDefaultStore(String defaultStore) {
+    this.defaultStore = defaultStore;
   }
 
-  public void setFvwt(String defaultFvwt) {
-    this.defaultFvwt = defaultFvwt;
+  public void setDefaultFormat(String defaultFormat) {
+    this.defaultFormat = defaultFormat;
+  }
+
+  public void setCsvKeyValueDelimiter(String csvKeyValueDelimiter) {
+    if (csvKeyValueDelimiter.length() != 1) {
+      throw new IllegalArgumentException("csvKeyValueDelimiter must be exactly 1 character");
+    }
+    this.csvKeyValueDelimiter = csvKeyValueDelimiter.charAt(0);
   }
 
-  public void setFormat(String defaultFvFormat) {
-    this.defaultFvFormat = defaultFvFormat;
+  public void setCsvFeatureSeparator(String csvFeatureSeparator) {
+    if (csvFeatureSeparator.length() != 1) {
+      throw new IllegalArgumentException("csvFeatureSeparator must be exactly 1 character");
+    }
+    this.csvFeatureSeparator = csvFeatureSeparator.charAt(0);
   }
 
   @Override
@@ -95,35 +121,62 @@ public class LTRFeatureLoggerTransformerFactory extends TransformerFactory {
   }
 
   @Override
-  public DocTransformer create(String name, SolrParams params,
+  public DocTransformer create(String name, SolrParams localparams,
       SolrQueryRequest req) {
 
     // Hint to enable feature vector cache since we are requesting features
     SolrQueryRequestContextUtils.setIsExtractingFeatures(req);
 
     // Communicate which feature store we are requesting features for
-    SolrQueryRequestContextUtils.setFvStoreName(req, params.get(FV_STORE, defaultFvStore));
+    SolrQueryRequestContextUtils.setFvStoreName(req, localparams.get(FV_STORE, defaultStore));
 
     // Create and supply the feature logger to be used
     SolrQueryRequestContextUtils.setFeatureLogger(req,
-        FeatureLogger.createFeatureLogger(
-            params.get(FV_RESPONSE_WRITER, defaultFvwt),
-            params.get(FV_FORMAT, defaultFvFormat)));
+        createFeatureLogger(
+            localparams.get(FV_FORMAT, defaultFormat)));
+
+    return new FeatureTransformer(name, localparams, req);
+  }
 
-    return new FeatureTransformer(name, params, req);
+  /**
+   * returns a FeatureLogger that logs the features
+   * 'featureFormat' param: 'dense' will write features in dense format,
+   * 'sparse' will write the features in sparse format, null or empty will
+   * default to 'sparse'
+   *
+   *
+   * @return a feature logger for the format specified.
+   */
+  private FeatureLogger createFeatureLogger(String featureFormat) {
+    final FeatureLogger.FeatureFormat f;
+    if (featureFormat == null || featureFormat.isEmpty() ||
+        featureFormat.equals("sparse")) {
+      f = FeatureLogger.FeatureFormat.SPARSE;
+    }
+    else if (featureFormat.equals("dense")) {
+      f = FeatureLogger.FeatureFormat.DENSE;
+    }
+    else {
+      f = FeatureLogger.FeatureFormat.SPARSE;
+      log.warn("unknown feature logger feature format {}", featureFormat);
+    }
+    if (fvCacheName == null) {
+      throw new IllegalArgumentException("a fvCacheName must be configured");
+    }
+    return new CSVFeatureLogger(fvCacheName, f, csvKeyValueDelimiter, csvFeatureSeparator);
   }
 
   class FeatureTransformer extends DocTransformer {
 
     final private String name;
-    final private SolrParams params;
+    final private SolrParams localparams;
     final private SolrQueryRequest req;
 
     private List<LeafReaderContext> leafContexts;
     private SolrIndexSearcher searcher;
     private LTRScoringQuery scoringQuery;
     private LTRScoringQuery.ModelWeight modelWeight;
-    private FeatureLogger<?> featureLogger;
+    private FeatureLogger featureLogger;
     private boolean docsWereNotReranked;
 
     /**
@@ -131,10 +184,10 @@ public class LTRFeatureLoggerTransformerFactory extends TransformerFactory {
      *          Name of the field to be added in a document representing the
      *          feature vectors
      */
-    public FeatureTransformer(String name, SolrParams params,
+    public FeatureTransformer(String name, SolrParams localparams,
         SolrQueryRequest req) {
       this.name = name;
-      this.params = params;
+      this.localparams = localparams;
       this.req = req;
     }
 
@@ -178,7 +231,7 @@ public class LTRFeatureLoggerTransformerFactory extends TransformerFactory {
               featureStoreName, store.getFeatures());
 
           scoringQuery = new LTRScoringQuery(lm,
-              LTRQParserPlugin.extractEFIParams(params),
+              LTRQParserPlugin.extractEFIParams(localparams),
               true,
               threadManager); // request feature weights to be created for all features
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c8542b2b/solr/contrib/ltr/src/java/overview.html
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/java/overview.html b/solr/contrib/ltr/src/java/overview.html
index d1a22f0..b0802d2 100644
--- a/solr/contrib/ltr/src/java/overview.html
+++ b/solr/contrib/ltr/src/java/overview.html
@@ -28,7 +28,7 @@ that sophisticated models can make more nuanced ranking decisions than standard
 functions like TF-IDF or BM25.
 </p>
 <p>
-This module allows to plug a reranking component directly into Solr, enabling users
+This module allows to plug a reranking model directly into Solr, enabling users
 to easily build their own learning to rank systems and access the rich
 matching features readily available in Solr. It also provides tools to perform
 feature engineering and feature extraction.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c8542b2b/solr/contrib/ltr/src/test-files/solr/collection1/conf/solrconfig-ltr.xml
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test-files/solr/collection1/conf/solrconfig-ltr.xml b/solr/contrib/ltr/src/test-files/solr/collection1/conf/solrconfig-ltr.xml
index d6f8d8e..1e1a618 100644
--- a/solr/contrib/ltr/src/test-files/solr/collection1/conf/solrconfig-ltr.xml
+++ b/solr/contrib/ltr/src/test-files/solr/collection1/conf/solrconfig-ltr.xml
@@ -35,8 +35,9 @@
   in the response. The name of the field we will be the the name of the transformer
   enclosed between brackets (in this case [fv]). In order to get the feature
   vector you will have to specify that you want the field (e.g., fl="*,[fv]) -->
- <transformer name="fv"
-  class="org.apache.solr.ltr.response.transform.LTRFeatureLoggerTransformerFactory" />
+ <transformer name="fv" class="org.apache.solr.ltr.response.transform.LTRFeatureLoggerTransformerFactory">
+   <str name="fvCacheName">QUERY_DOC_FV</str>
+ </transformer>
 
  <updateHandler class="solr.DirectUpdateHandler2">
   <autoCommit>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c8542b2b/solr/contrib/ltr/src/test-files/solr/collection1/conf/solrconfig-ltr_Th10_10.xml
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test-files/solr/collection1/conf/solrconfig-ltr_Th10_10.xml b/solr/contrib/ltr/src/test-files/solr/collection1/conf/solrconfig-ltr_Th10_10.xml
index 51971f7..b41bce1 100644
--- a/solr/contrib/ltr/src/test-files/solr/collection1/conf/solrconfig-ltr_Th10_10.xml
+++ b/solr/contrib/ltr/src/test-files/solr/collection1/conf/solrconfig-ltr_Th10_10.xml
@@ -39,8 +39,9 @@
   in the response. The name of the field we will be the the name of the transformer
   enclosed between brackets (in this case [fv]). In order to get the feature
   vector you will have to specify that you want the field (e.g., fl="*,[fv]) -->
- <transformer name="fv"
-  class="org.apache.solr.ltr.response.transform.LTRFeatureLoggerTransformerFactory" />
+ <transformer name="fv" class="org.apache.solr.ltr.response.transform.LTRFeatureLoggerTransformerFactory">
+   <str name="fvCacheName">QUERY_DOC_FV</str>
+ </transformer>
 
  <updateHandler class="solr.DirectUpdateHandler2">
   <autoCommit>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c8542b2b/solr/contrib/ltr/src/test-files/solr/collection1/conf/solrconfig-multiseg.xml
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test-files/solr/collection1/conf/solrconfig-multiseg.xml b/solr/contrib/ltr/src/test-files/solr/collection1/conf/solrconfig-multiseg.xml
index 5abff6f..b34be8f 100644
--- a/solr/contrib/ltr/src/test-files/solr/collection1/conf/solrconfig-multiseg.xml
+++ b/solr/contrib/ltr/src/test-files/solr/collection1/conf/solrconfig-multiseg.xml
@@ -22,6 +22,13 @@
  <!-- Query parser used to rerank top docs with a provided model -->
  <queryParser name="ltr" class="org.apache.solr.ltr.search.LTRQParserPlugin" />
 
+ <query>
+  <filterCache class="solr.FastLRUCache" size="4096"
+   initialSize="2048" autowarmCount="0" />
+  <cache name="QUERY_DOC_FV" class="solr.search.LRUCache" size="4096"
+   initialSize="2048" autowarmCount="4096" regenerator="solr.search.NoOpRegenerator" />
+ </query>
+
  <maxBufferedDocs>1</maxBufferedDocs>
  <mergePolicyFactory class="org.apache.solr.index.TieredMergePolicyFactory">
   <int name="maxMergeAtOnce">10</int>
@@ -32,8 +39,9 @@
   in the response. The name of the field we will be the the name of the transformer 
   enclosed between brackets (in this case [fv]). In order to get the feature 
   vector you will have to specify that you want the field (e.g., fl="*,[fv]) -->
- <transformer name="features"
-  class="org.apache.solr.ltr.response.transform.LTRFeatureLoggerTransformerFactory" />
+ <transformer name="features" class="org.apache.solr.ltr.response.transform.LTRFeatureLoggerTransformerFactory">
+   <str name="fvCacheName">QUERY_DOC_FV</str>
+ </transformer>
 
  <updateHandler class="solr.DirectUpdateHandler2">
   <autoCommit>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c8542b2b/solr/contrib/ltr/src/test/org/apache/solr/ltr/FeatureLoggerTestUtils.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/FeatureLoggerTestUtils.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/FeatureLoggerTestUtils.java
new file mode 100644
index 0000000..f2afd9c
--- /dev/null
+++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/FeatureLoggerTestUtils.java
@@ -0,0 +1,44 @@
+/*
+ * 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.ltr;
+
+public class FeatureLoggerTestUtils {
+
+  public static String toFeatureVector(String ... keysAndValues) {
+    return toFeatureVector(
+        CSVFeatureLogger.DEFAULT_KEY_VALUE_SEPARATOR,
+        CSVFeatureLogger.DEFAULT_FEATURE_SEPARATOR,
+        keysAndValues);
+  }
+
+  public static String toFeatureVector(char keyValueSeparator, char featureSeparator,
+      String ... keysAndValues) {
+    StringBuilder sb = new StringBuilder(keysAndValues.length/2 * 3);
+    for (int ii = 0; ii+1 < keysAndValues.length; ii += 2) {
+        sb.append(keysAndValues[ii])
+        .append(keyValueSeparator)
+        .append(keysAndValues[ii+1])
+        .append(featureSeparator);
+    }
+
+    final String features = (sb.length() > 0 ?
+        sb.substring(0, sb.length() - 1) : "");
+
+    return features;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c8542b2b/solr/contrib/ltr/src/test/org/apache/solr/ltr/TestLTROnSolrCloud.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/TestLTROnSolrCloud.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/TestLTROnSolrCloud.java
index 2e01a64..2bef197 100644
--- a/solr/contrib/ltr/src/test/org/apache/solr/ltr/TestLTROnSolrCloud.java
+++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/TestLTROnSolrCloud.java
@@ -92,6 +92,15 @@ public class TestLTROnSolrCloud extends TestRerankBase {
     assertEquals("3", queryResponse.getResults().get(2).get("id").toString());
     assertEquals("4", queryResponse.getResults().get(3).get("id").toString());
 
+    final String result0_features= FeatureLoggerTestUtils.toFeatureVector(
+        "powpularityS","64.0", "c3","2.0");
+    final String result1_features= FeatureLoggerTestUtils.toFeatureVector(
+        "powpularityS","49.0", "c3","2.0");
+    final String result2_features= FeatureLoggerTestUtils.toFeatureVector(
+        "powpularityS","36.0", "c3","2.0");
+    final String result3_features= FeatureLoggerTestUtils.toFeatureVector(
+        "powpularityS","25.0", "c3","2.0");
+
     // Test re-rank and feature vectors returned
     query.setFields("*,score,features:[fv]");
     query.add("rq", "{!ltr model=powpularityS-model reRankDocs=8}");
@@ -99,16 +108,16 @@ public class TestLTROnSolrCloud extends TestRerankBase {
         solrCluster.getSolrClient().query(COLLECTION,query);
     assertEquals(8, queryResponse.getResults().getNumFound());
     assertEquals("8", queryResponse.getResults().get(0).get("id").toString());
-    assertEquals("powpularityS:64.0;c3:2.0",
+    assertEquals(result0_features,
         queryResponse.getResults().get(0).get("features").toString());
     assertEquals("7", queryResponse.getResults().get(1).get("id").toString());
-    assertEquals("powpularityS:49.0;c3:2.0",
+    assertEquals(result1_features,
         queryResponse.getResults().get(1).get("features").toString());
     assertEquals("6", queryResponse.getResults().get(2).get("id").toString());
-    assertEquals("powpularityS:36.0;c3:2.0",
+    assertEquals(result2_features,
         queryResponse.getResults().get(2).get("features").toString());
     assertEquals("5", queryResponse.getResults().get(3).get("id").toString());
-    assertEquals("powpularityS:25.0;c3:2.0",
+    assertEquals(result3_features,
         queryResponse.getResults().get(3).get("features").toString());
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c8542b2b/solr/contrib/ltr/src/test/org/apache/solr/ltr/TestSelectiveWeightCreation.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/TestSelectiveWeightCreation.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/TestSelectiveWeightCreation.java
index 68961d2..b9b3d63 100644
--- a/solr/contrib/ltr/src/test/org/apache/solr/ltr/TestSelectiveWeightCreation.java
+++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/TestSelectiveWeightCreation.java
@@ -215,6 +215,11 @@ public class TestSelectiveWeightCreation extends TestRerankBase {
   @Test
   public void testSelectiveWeightsRequestFeaturesFromDifferentStore() throws Exception {
 
+    final String docs0fv = FeatureLoggerTestUtils.toFeatureVector(
+        "matchedTitle","1.0", "titlePhraseMatch","0.40254828");
+    final String docs0fv_fstore4= FeatureLoggerTestUtils.toFeatureVector(
+        "popularity","3.0", "originalScore","1.0");
+
     final SolrQuery query = new SolrQuery();
     query.setQuery("*:*");
     query.add("fl", "*,score");
@@ -225,7 +230,7 @@ public class TestSelectiveWeightCreation extends TestRerankBase {
     assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='1'");
     assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/id=='3'");
     assertJQ("/query" + query.toQueryString(), "/response/docs/[2]/id=='4'");
-    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/fv=='matchedTitle:1.0;titlePhraseMatch:0.40254828'"); // extract all features in default store
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/fv=='"+docs0fv+"'"); // extract all features in default store
 
     query.remove("fl");
     query.remove("rq");
@@ -235,7 +240,7 @@ public class TestSelectiveWeightCreation extends TestRerankBase {
 
     assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='1'");
     assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/score==0.999");
-    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/fv=='popularity:3.0;originalScore:1.0'"); // extract all features from fstore4
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/fv=='"+docs0fv_fstore4+"'"); // extract all features from fstore4
 
 
     query.remove("fl");
@@ -245,7 +250,7 @@ public class TestSelectiveWeightCreation extends TestRerankBase {
     query.add("fl", "fv:[fv store=fstore4 efi.myPop=3]");
     assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='1'"); // score using fstore2 used by externalmodelstore
     assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/score==0.7992");
-    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/fv=='popularity:3.0;originalScore:1.0'"); // extract all features from fstore4
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/fv=='"+docs0fv_fstore4+"'"); // extract all features from fstore4
   }
 }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c8542b2b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestExternalFeatures.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestExternalFeatures.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestExternalFeatures.java
index 8c00758..15b7633 100644
--- a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestExternalFeatures.java
+++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestExternalFeatures.java
@@ -17,6 +17,7 @@
 package org.apache.solr.ltr.feature;
 
 import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.ltr.FeatureLoggerTestUtils;
 import org.apache.solr.ltr.TestRerankBase;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -103,6 +104,9 @@ public class TestExternalFeatures extends TestRerankBase {
     query.setQuery("*:*");
     query.add("rows", "1");
 
+    final String docs0fv_sparse_csv = FeatureLoggerTestUtils.toFeatureVector(
+        "confidence","2.3", "originalScore","1.0");
+
     // Features we're extracting depend on external feature info not passed in
     query.add("fl", "[fv]");
     assertJQ("/query" + query.toQueryString(), "/error/msg=='Exception from createWeight for SolrFeature [name=matchedTitle, params={q={!terms f=title}${user_query}}] SolrFeatureWeight requires efi parameter that was not passed in request.'");
@@ -110,13 +114,13 @@ public class TestExternalFeatures extends TestRerankBase {
     // Adding efi in features section should make it work
     query.remove("fl");
     query.add("fl", "score,fvalias:[fv store=fstore2 efi.myconf=2.3]");
-    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/fvalias=='confidence:2.3;originalScore:1.0'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/fvalias=='"+docs0fv_sparse_csv+"'");
 
     // Adding efi in transformer + rq should still use the transformer's params for feature extraction
     query.remove("fl");
     query.add("fl", "score,fvalias:[fv store=fstore2 efi.myconf=2.3]");
     query.add("rq", "{!ltr reRankDocs=3 model=externalmodel efi.user_query=w3}");
-    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/fvalias=='confidence:2.3;originalScore:1.0'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/fvalias=='"+docs0fv_sparse_csv+"'");
   }
 
   @Test
@@ -128,7 +132,7 @@ public class TestExternalFeatures extends TestRerankBase {
     // Efi is explicitly not required, so we do not score the feature
     query.remove("fl");
     query.add("fl", "fvalias:[fv store=fstore2]");
-    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/fvalias=='originalScore:0.0'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/fvalias=='"+FeatureLoggerTestUtils.toFeatureVector("originalScore","0.0")+"'");
   }
 
   @Test
@@ -140,7 +144,7 @@ public class TestExternalFeatures extends TestRerankBase {
     // Efi is explicitly not required, so we do not score the feature
     query.remove("fl");
     query.add("fl", "fvalias:[fv store=fstore3]");
-    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/fvalias=='originalScore:0.0'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/fvalias=='"+FeatureLoggerTestUtils.toFeatureVector("originalScore","0.0")+"'");
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c8542b2b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestExternalValueFeatures.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestExternalValueFeatures.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestExternalValueFeatures.java
index bc073cb..fc0ade2 100644
--- a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestExternalValueFeatures.java
+++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestExternalValueFeatures.java
@@ -17,6 +17,7 @@
 package org.apache.solr.ltr.feature;
 
 import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.ltr.FeatureLoggerTestUtils;
 import org.apache.solr.ltr.TestRerankBase;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -55,12 +56,11 @@ public class TestExternalValueFeatures extends TestRerankBase {
     query.setQuery("*:*");
     query.add("fl", "*,score,features:[fv]");
     query.add("rows", "3");
-    query.add("fl", "[fv]");
     query.add("rq", "{!ltr reRankDocs=3 model=external_model_binary_feature efi.user_device_tablet=1}");
 
     assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='1'");
     assertJQ("/query" + query.toQueryString(),
-        "/response/docs/[0]/features=='user_device_tablet:1.0'");
+        "/response/docs/[0]/features=='"+FeatureLoggerTestUtils.toFeatureVector("user_device_tablet","1.0")+"'");
     assertJQ("/query" + query.toQueryString(),
         "/response/docs/[0]/score==65.0");
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c8542b2b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFeatureLogging.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFeatureLogging.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFeatureLogging.java
index 14e2903..f18c6bf 100644
--- a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFeatureLogging.java
+++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFeatureLogging.java
@@ -17,6 +17,7 @@
 package org.apache.solr.ltr.feature;
 
 import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.ltr.FeatureLoggerTestUtils;
 import org.apache.solr.ltr.TestRerankBase;
 import org.apache.solr.ltr.model.LinearModel;
 import org.apache.solr.ltr.store.FeatureStore;
@@ -55,6 +56,13 @@ public class TestFeatureLogging extends TestRerankBase {
         "c1", "c2", "c3"}, "test1",
         "{\"weights\":{\"c1\":1.0,\"c2\":1.0,\"c3\":1.0}}");
 
+    final String docs0fv_sparse_csv = FeatureLoggerTestUtils.toFeatureVector(
+        "c1","1.0",
+        "c2","2.0",
+        "c3","3.0",
+        "pop","2.0",
+        "yesmatch","1.0");
+
     final SolrQuery query = new SolrQuery();
     query.setQuery("title:bloomberg");
     query.add("fl", "title,description,id,popularity,[fv]");
@@ -65,7 +73,7 @@ public class TestFeatureLogging extends TestRerankBase {
     restTestHarness.query("/query" + query.toQueryString());
     assertJQ(
         "/query" + query.toQueryString(),
-        "/response/docs/[0]/=={'title':'bloomberg bloomberg ', 'description':'bloomberg','id':'7', 'popularity':2,  '[fv]':'c1:1.0;c2:2.0;c3:3.0;pop:2.0;yesmatch:1.0'}");
+        "/response/docs/[0]/=={'title':'bloomberg bloomberg ', 'description':'bloomberg','id':'7', 'popularity':2,  '[fv]':'"+docs0fv_sparse_csv+"'}");
 
     query.remove("fl");
     query.add("fl", "[fv]");
@@ -74,15 +82,7 @@ public class TestFeatureLogging extends TestRerankBase {
 
     restTestHarness.query("/query" + query.toQueryString());
     assertJQ("/query" + query.toQueryString(),
-        "/response/docs/[0]/=={'[fv]':'c1:1.0;c2:2.0;c3:3.0;pop:2.0;yesmatch:1.0'}");
-    query.remove("rq");
-
-    // set logging at false but still asking for feature, and it should work anyway
-    query.add("rq", "{!ltr reRankDocs=3 model=sum1}");
-    assertJQ("/query" + query.toQueryString(),
-        "/response/docs/[0]/=={'[fv]':'c1:1.0;c2:2.0;c3:3.0;pop:2.0;yesmatch:1.0'}");
-
-
+        "/response/docs/[0]/=={'[fv]':'"+docs0fv_sparse_csv+"'}");
   }
 
   @Test
@@ -108,24 +108,24 @@ public class TestFeatureLogging extends TestRerankBase {
     // No store specified, use default store for extraction
     query.add("fl", "fv:[fv]");
     assertJQ("/query" + query.toQueryString(),
-        "/response/docs/[0]/=={'fv':'defaultf1:1.0'}");
+        "/response/docs/[0]/=={'fv':'"+FeatureLoggerTestUtils.toFeatureVector("defaultf1","1.0")+"'}");
 
     // Store specified, use store for extraction
     query.remove("fl");
     query.add("fl", "fv:[fv store=store8]");
     assertJQ("/query" + query.toQueryString(),
-        "/response/docs/[0]/=={'fv':'store8f1:2.0'}");
+        "/response/docs/[0]/=={'fv':'"+FeatureLoggerTestUtils.toFeatureVector("store8f1","2.0")+"'}");
 
     // Store specified + model specified, use store for extraction
     query.add("rq", "{!ltr reRankDocs=3 model=store9m1}");
     assertJQ("/query" + query.toQueryString(),
-        "/response/docs/[0]/=={'fv':'store8f1:2.0'}");
+        "/response/docs/[0]/=={'fv':'"+FeatureLoggerTestUtils.toFeatureVector("store8f1","2.0")+"'}");
 
     // No store specified + model specified, use model store for extraction
     query.remove("fl");
     query.add("fl", "fv:[fv]");
     assertJQ("/query" + query.toQueryString(),
-        "/response/docs/[0]/=={'fv':'store9f1:3.0'}");
+        "/response/docs/[0]/=={'fv':'"+FeatureLoggerTestUtils.toFeatureVector("store9f1","3.0")+"'}");
   }
 
 
@@ -157,23 +157,16 @@ public class TestFeatureLogging extends TestRerankBase {
 
     query.add("rq", "{!ltr reRankDocs=3 model=sumgroup}");
 
-    restTestHarness.query("/query" + query.toQueryString());
-    assertJQ(
-        "/query" + query.toQueryString(),
-        "/grouped/title/groups/[0]/doclist/docs/[0]/=={'fv':'c1:1.0;c2:2.0;c3:3.0;pop:5.0'}");
+    final String docs0fv_sparse_csv = FeatureLoggerTestUtils.toFeatureVector(
+        "c1","1.0",
+        "c2","2.0",
+        "c3","3.0",
+        "pop","5.0");
 
-    query.remove("fl");
-    query.add("fl", "fv:[fv fvwt=json]");
     restTestHarness.query("/query" + query.toQueryString());
     assertJQ(
         "/query" + query.toQueryString(),
-        "/grouped/title/groups/[0]/doclist/docs/[0]/fv/=={'c1':1.0,'c2':2.0,'c3':3.0,'pop':5.0}");
-    query.remove("fl");
-    query.add("fl", "fv:[fv fvwt=json]");
-
-    assertJQ(
-        "/query" + query.toQueryString(),
-        "/grouped/title/groups/[0]/doclist/docs/[0]/fv/=={'c1':1.0,'c2':2.0,'c3':3.0,'pop':5.0}");
+        "/grouped/title/groups/[0]/doclist/docs/[0]/=={'fv':'"+docs0fv_sparse_csv+"'}");
   }
 
   @Test
@@ -187,68 +180,46 @@ public class TestFeatureLogging extends TestRerankBase {
         "match"}, "test4",
         "{\"weights\":{\"match\":1.0}}");
 
-    //json - no feature format check (default to sparse)
+    final String docs0fv_sparse_csv = FeatureLoggerTestUtils.toFeatureVector("match", "1.0", "c4", "1.0");
+    final String docs1fv_sparse_csv = FeatureLoggerTestUtils.toFeatureVector("c4", "1.0");
+
+    final String docs0fv_dense_csv  = FeatureLoggerTestUtils.toFeatureVector("match", "1.0", "c4", "1.0");
+    final String docs1fv_dense_csv  = FeatureLoggerTestUtils.toFeatureVector("match", "0.0", "c4", "1.0");
+
     final SolrQuery query = new SolrQuery();
     query.setQuery("title:bloomberg");
     query.add("rows", "10");
-    query.add("fl", "*,score,fv:[fv store=test4 fvwt=json]");
     query.add("rq", "{!ltr reRankDocs=10 model=sum4}");
-    assertJQ(
-        "/query" + query.toQueryString(),
-        "/response/docs/[0]/fv/=={'match':1.0,'c4':1.0}");
-    assertJQ(
-        "/query" + query.toQueryString(),
-        "/response/docs/[1]/fv/=={'c4':1.0}");
-
-    //json - sparse feature format check
-    query.remove("fl");
-    query.add("fl", "*,score,fv:[fv store=test4 format=sparse fvwt=json]");
-    assertJQ(
-        "/query" + query.toQueryString(),
-        "/response/docs/[0]/fv/=={'match':1.0,'c4':1.0}");
-    assertJQ(
-        "/query" + query.toQueryString(),
-        "/response/docs/[1]/fv/=={'c4':1.0}");
-
-    //json - dense feature format check
-    query.remove("fl");
-    query.add("fl", "*,score,fv:[fv store=test4 format=dense fvwt=json]");
-    assertJQ(
-        "/query" + query.toQueryString(),
-        "/response/docs/[0]/fv/=={'match':1.0,'c4':1.0}");
-    assertJQ(
-        "/query" + query.toQueryString(),
-        "/response/docs/[1]/fv/=={'match':0.0,'c4':1.0}");
 
     //csv - no feature format check (default to sparse)
     query.remove("fl");
-    query.add("fl", "*,score,fv:[fv store=test4 fvwt=csv]");
+    query.add("fl", "*,score,fv:[fv store=test4]");
     assertJQ(
         "/query" + query.toQueryString(),
-        "/response/docs/[0]/fv/=='match:1.0;c4:1.0'");
+        "/response/docs/[0]/fv/=='"+docs0fv_sparse_csv+"'");
     assertJQ(
         "/query" + query.toQueryString(),
-        "/response/docs/[1]/fv/=='c4:1.0'");
+        "/response/docs/[1]/fv/=='"+docs1fv_sparse_csv+"'");
 
     //csv - sparse feature format check
     query.remove("fl");
-    query.add("fl", "*,score,fv:[fv store=test4 format=sparse fvwt=csv]");
+    query.add("fl", "*,score,fv:[fv store=test4 format=sparse]");
     assertJQ(
         "/query" + query.toQueryString(),
-        "/response/docs/[0]/fv/=='match:1.0;c4:1.0'");
+        "/response/docs/[0]/fv/=='"+docs0fv_sparse_csv+"'");
     assertJQ(
         "/query" + query.toQueryString(),
-        "/response/docs/[1]/fv/=='c4:1.0'");
+        "/response/docs/[1]/fv/=='"+docs1fv_sparse_csv+"'");
 
     //csv - dense feature format check
     query.remove("fl");
-    query.add("fl", "*,score,fv:[fv store=test4 format=dense fvwt=csv]");
+    query.add("fl", "*,score,fv:[fv store=test4 format=dense]");
     assertJQ(
         "/query" + query.toQueryString(),
-        "/response/docs/[0]/fv/=='match:1.0;c4:1.0'");
+        "/response/docs/[0]/fv/=='"+docs0fv_dense_csv+"'");
     assertJQ(
         "/query" + query.toQueryString(),
-        "/response/docs/[1]/fv/=='match:0.0;c4:1.0'");
+        "/response/docs/[1]/fv/=='"+docs1fv_dense_csv+"'");
   }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c8542b2b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFieldValueFeature.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFieldValueFeature.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFieldValueFeature.java
index 9574273..8295403 100644
--- a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFieldValueFeature.java
+++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFieldValueFeature.java
@@ -17,6 +17,7 @@
 package org.apache.solr.ltr.feature;
 
 import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.ltr.FeatureLoggerTestUtils;
 import org.apache.solr.ltr.TestRerankBase;
 import org.apache.solr.ltr.model.LinearModel;
 import org.junit.AfterClass;
@@ -116,7 +117,7 @@ public class TestFieldValueFeature extends TestRerankBase {
     query.add("fl", "[fv]");
     assertJQ("/query" + query.toQueryString(), "/response/numFound/==1");
     assertJQ("/query" + query.toQueryString(),
-            "/response/docs/[0]/=={'[fv]':'popularity:"+FIELD_VALUE_FEATURE_DEFAULT_VAL+"'}");
+            "/response/docs/[0]/=={'[fv]':'"+FeatureLoggerTestUtils.toFeatureVector("popularity",Float.toString(FIELD_VALUE_FEATURE_DEFAULT_VAL))+"'}");
 
   }
 
@@ -145,7 +146,7 @@ public class TestFieldValueFeature extends TestRerankBase {
     query.add("fl", "[fv]");
     assertJQ("/query" + query.toQueryString(), "/response/numFound/==1");
     assertJQ("/query" + query.toQueryString(),
-            "/response/docs/[0]/=={'[fv]':'popularity42:42.0'}");
+            "/response/docs/[0]/=={'[fv]':'"+FeatureLoggerTestUtils.toFeatureVector("popularity42","42.0")+"'}");
 
   }
 
@@ -165,7 +166,7 @@ public class TestFieldValueFeature extends TestRerankBase {
     query.add("fl", "[fv]");
     assertJQ("/query" + query.toQueryString(), "/response/numFound/==1");
     assertJQ("/query" + query.toQueryString(),
-            "/response/docs/[0]/=={'[fv]':'not-existing-field:"+FIELD_VALUE_FEATURE_DEFAULT_VAL+"'}");
+            "/response/docs/[0]/=={'[fv]':'"+FeatureLoggerTestUtils.toFeatureVector("not-existing-field",Float.toString(FIELD_VALUE_FEATURE_DEFAULT_VAL))+"'}");
 
   }
 
@@ -183,7 +184,7 @@ public class TestFieldValueFeature extends TestRerankBase {
     query.add("rq", "{!ltr model=trendy-model reRankDocs=4}");
     query.add("fl", "[fv]");
     assertJQ("/query" + query.toQueryString(),
-            "/response/docs/[0]/=={'[fv]':'trendy:0.0'}");
+            "/response/docs/[0]/=={'[fv]':'"+FeatureLoggerTestUtils.toFeatureVector("trendy","0.0")+"'}");
 
 
     query = new SolrQuery();
@@ -191,7 +192,7 @@ public class TestFieldValueFeature extends TestRerankBase {
     query.add("rq", "{!ltr model=trendy-model reRankDocs=4}");
     query.add("fl", "[fv]");
     assertJQ("/query" + query.toQueryString(),
-            "/response/docs/[0]/=={'[fv]':'trendy:1.0'}");
+            "/response/docs/[0]/=={'[fv]':'"+FeatureLoggerTestUtils.toFeatureVector("trendy","1.0")+"'}");
 
     // check default value is false
     query = new SolrQuery();
@@ -199,7 +200,7 @@ public class TestFieldValueFeature extends TestRerankBase {
     query.add("rq", "{!ltr model=trendy-model reRankDocs=4}");
     query.add("fl", "[fv]");
     assertJQ("/query" + query.toQueryString(),
-            "/response/docs/[0]/=={'[fv]':'trendy:0.0'}");
+            "/response/docs/[0]/=={'[fv]':'"+FeatureLoggerTestUtils.toFeatureVector("trendy","0.0")+"'}");
 
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c8542b2b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFilterSolrFeature.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFilterSolrFeature.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFilterSolrFeature.java
index 14baefa..a6a80bd 100644
--- a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFilterSolrFeature.java
+++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestFilterSolrFeature.java
@@ -17,6 +17,7 @@
 package org.apache.solr.ltr.feature;
 
 import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.ltr.FeatureLoggerTestUtils;
 import org.apache.solr.ltr.TestRerankBase;
 import org.apache.solr.ltr.model.LinearModel;
 import org.apache.solr.ltr.store.rest.ManagedFeatureStore;
@@ -96,10 +97,13 @@ public class TestFilterSolrFeature extends TestRerankBase {
     query.add("rq", "{!ltr reRankDocs=4 model=fqmodel efi.user_query=w2}");
     query.add("fl", "fv:[fv]");
 
+    final String docs0fv_sparse_csv= FeatureLoggerTestUtils.toFeatureVector(
+        "matchedTitle","1.0", "popularity","3.0");
+
     assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='2'");
     assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/id=='1'");
     assertJQ("/query" + query.toQueryString(), "/response/docs/[2]/id=='3'");
-    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/fv=='matchedTitle:1.0;popularity:3.0'");
+    assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/fv=='"+docs0fv_sparse_csv+"'");
   }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c8542b2b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestNoMatchSolrFeature.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestNoMatchSolrFeature.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestNoMatchSolrFeature.java
index 5712687..004e314 100644
--- a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestNoMatchSolrFeature.java
+++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestNoMatchSolrFeature.java
@@ -20,6 +20,7 @@ import java.util.ArrayList;
 import java.util.Map;
 
 import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.ltr.FeatureLoggerTestUtils;
 import org.apache.solr.ltr.TestRerankBase;
 import org.apache.solr.ltr.model.LinearModel;
 import org.apache.solr.ltr.model.MultipleAdditiveTreesModel;
@@ -104,12 +105,11 @@ public class TestNoMatchSolrFeature extends TestRerankBase {
     final Double doc0Score = (Double) ((Map<String,Object>) ((ArrayList<Object>) ((Map<String,Object>) jsonParse
         .get("response")).get("docs")).get(0)).get("score");
 
-
     assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='1'");
     assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/score=="
         + (doc0Score * 1.1));
     assertJQ("/query" + query.toQueryString(),
-        "/response/docs/[0]/fv=='yesmatchfeature:" + doc0Score + "'");
+        "/response/docs/[0]/fv=='"+FeatureLoggerTestUtils.toFeatureVector("yesmatchfeature", doc0Score.toString())+"'");
     assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/id=='2'");
     assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/score==0.0");
     assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/fv==''");
@@ -144,7 +144,7 @@ public class TestNoMatchSolrFeature extends TestRerankBase {
 
     assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/score==0.0");
     assertJQ("/query" + query.toQueryString(),
-        "/response/docs/[0]/fv=='yesmatchfeature:" + doc0Score + "'");
+        "/response/docs/[0]/fv=='"+FeatureLoggerTestUtils.toFeatureVector("yesmatchfeature", doc0Score.toString())+"'");
     assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/score==0.0");
     assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/fv==''");
     assertJQ("/query" + query.toQueryString(), "/response/docs/[2]/score==0.0");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c8542b2b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestOriginalScoreFeature.java
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestOriginalScoreFeature.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestOriginalScoreFeature.java
index e525891..48662e6 100644
--- a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestOriginalScoreFeature.java
+++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestOriginalScoreFeature.java
@@ -20,6 +20,7 @@ import java.util.ArrayList;
 import java.util.Map;
 
 import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.ltr.FeatureLoggerTestUtils;
 import org.apache.solr.ltr.TestRerankBase;
 import org.apache.solr.ltr.model.LinearModel;
 import org.junit.AfterClass;
@@ -132,17 +133,17 @@ public class TestOriginalScoreFeature extends TestRerankBase {
     assertJQ("/query" + query.toQueryString(), "/response/numFound/==4");
     assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='1'");
     assertJQ("/query" + query.toQueryString(),
-        "/response/docs/[0]/fv=='origScore:" + doc0Score + ";c2:2.0'");
+        "/response/docs/[0]/fv=='" + FeatureLoggerTestUtils.toFeatureVector("origScore", doc0Score, "c2", "2.0")+"'");
     assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/id=='8'");
 
     assertJQ("/query" + query.toQueryString(),
-        "/response/docs/[1]/fv=='origScore:" + doc1Score + ";c2:2.0'");
+        "/response/docs/[1]/fv=='" + FeatureLoggerTestUtils.toFeatureVector("origScore", doc1Score, "c2", "2.0")+"'");
     assertJQ("/query" + query.toQueryString(), "/response/docs/[2]/id=='6'");
     assertJQ("/query" + query.toQueryString(),
-        "/response/docs/[2]/fv=='origScore:" + doc2Score + ";c2:2.0'");
+        "/response/docs/[2]/fv=='" + FeatureLoggerTestUtils.toFeatureVector("origScore", doc2Score, "c2", "2.0")+"'");
     assertJQ("/query" + query.toQueryString(), "/response/docs/[3]/id=='7'");
     assertJQ("/query" + query.toQueryString(),
-        "/response/docs/[3]/fv=='origScore:" + doc3Score + ";c2:2.0'");
+        "/response/docs/[3]/fv=='" + FeatureLoggerTestUtils.toFeatureVector("origScore", doc3Score, "c2", "2.0")+"'");
   }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c8542b2b/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml b/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml
index e4b0526..990a23f 100644
--- a/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml
+++ b/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml
@@ -81,6 +81,9 @@
   <lib dir="${solr.install.dir:../../../..}/contrib/langid/lib/" regex=".*\.jar" />
   <lib dir="${solr.install.dir:../../../..}/dist/" regex="solr-langid-\d.*\.jar" />
 
+  <lib dir="${solr.install.dir:../../../..}/contrib/ltr/lib/" regex=".*\.jar" />
+  <lib dir="${solr.install.dir:../../../..}/dist/" regex="solr-ltr-\d.*\.jar" />
+
   <lib dir="${solr.install.dir:../../../..}/contrib/velocity/lib" regex=".*\.jar" />
   <lib dir="${solr.install.dir:../../../..}/dist/" regex="solr-velocity-\d.*\.jar" />
 
@@ -504,6 +507,23 @@
                         showItems="32" />
       -->
 
+    <!-- Feature Values Cache
+
+         Cache used by the Learning To Rank (LTR) contrib module.
+
+         You will need to set the solr.ltr.enabled system property
+         when running solr to run with ltr enabled:
+           -Dsolr.ltr.enabled=true
+
+         https://cwiki.apache.org/confluence/display/solr/Result+Reranking
+      -->
+    <cache enable="${solr.ltr.enabled:false}" name="QUERY_DOC_FV"
+           class="solr.search.LRUCache"
+           size="4096"
+           initialSize="2048"
+           autowarmCount="4096"
+           regenerator="solr.search.NoOpRegenerator" />
+
     <!-- Custom Cache
 
          Example of a generic cache.  These caches may be accessed by
@@ -1586,8 +1606,19 @@
      <valueSourceParser name="myfunc" 
                         class="com.mycompany.MyValueSourceParser" />
     -->
-    
-  
+
+  <!--  LTR query parser
+
+        You will need to set the solr.ltr.enabled system property
+        when running solr to run with ltr enabled:
+          -Dsolr.ltr.enabled=true
+
+        https://cwiki.apache.org/confluence/display/solr/Result+Reranking
+
+        Query parser is used to rerank top docs with a provided model
+    -->
+  <queryParser enable="${solr.ltr.enabled:false}" name="ltr" class="org.apache.solr.ltr.search.LTRQParserPlugin"/>
+
   <!-- Document Transformers
        http://wiki.apache.org/solr/DocTransformers
     -->
@@ -1611,5 +1642,22 @@
       EditorialMarkerFactory will do exactly that:
      <transformer name="qecBooster" class="org.apache.solr.response.transform.EditorialMarkerFactory" />
     -->
-    
+
+    <!--
+      LTR Transformer will encode the document features in the response. For each document the transformer
+      will add the features as an extra field in the response. The name of the field will be the
+      name of the transformer enclosed between brackets (in this case [features]).
+      In order to get the feature vector you will have to specify that you
+      want the field (e.g., fl="*,[features])
+
+      You will need to set the solr.ltr.enabled system property
+      when running solr to run with ltr enabled:
+        -Dsolr.ltr.enabled=true
+
+      https://cwiki.apache.org/confluence/display/solr/Result+Reranking
+      -->
+    <transformer enable="${solr.ltr.enabled:false}" name="features" class="org.apache.solr.ltr.response.transform.LTRFeatureLoggerTransformerFactory">
+      <str name="fvCacheName">QUERY_DOC_FV</str>
+    </transformer>
+
 </config>


[39/44] lucene-solr:jira/solr-9854: SOLR-4735 Remove stray imports.

Posted by ab...@apache.org.
SOLR-4735 Remove stray imports.


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

Branch: refs/heads/jira/solr-9854
Commit: aed34a40bc47d29eb87d50f162c187b6db0903b4
Parents: adfb4a5
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Tue Dec 20 09:30:03 2016 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Tue Dec 20 09:30:03 2016 +0100

----------------------------------------------------------------------
 solr/core/src/java/org/apache/solr/metrics/SolrMetricProducer.java | 1 -
 solr/core/src/java/org/apache/solr/metrics/SolrMetricReporter.java | 1 -
 solr/test-framework/src/java/org/apache/solr/util/TestHarness.java | 1 -
 3 files changed, 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aed34a40/solr/core/src/java/org/apache/solr/metrics/SolrMetricProducer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrMetricProducer.java b/solr/core/src/java/org/apache/solr/metrics/SolrMetricProducer.java
index b09a90a..472931f 100644
--- a/solr/core/src/java/org/apache/solr/metrics/SolrMetricProducer.java
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrMetricProducer.java
@@ -18,7 +18,6 @@ package org.apache.solr.metrics;
 
 import java.util.Collection;
 
-import com.codahale.metrics.MetricRegistry;
 import org.apache.solr.core.SolrInfoMBean;
 
 /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aed34a40/solr/core/src/java/org/apache/solr/metrics/SolrMetricReporter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrMetricReporter.java b/solr/core/src/java/org/apache/solr/metrics/SolrMetricReporter.java
index a36a1c3..ff2d3fc 100644
--- a/solr/core/src/java/org/apache/solr/metrics/SolrMetricReporter.java
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrMetricReporter.java
@@ -18,7 +18,6 @@ package org.apache.solr.metrics;
 
 import java.io.Closeable;
 
-import com.codahale.metrics.MetricRegistry;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.util.SolrPluginUtils;
 import org.apache.solr.util.plugin.PluginInfoInitialized;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/aed34a40/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java b/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
index 4e6f6d6..be8a24c 100644
--- a/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
+++ b/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
@@ -40,7 +40,6 @@ import org.apache.solr.core.NodeConfig;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.SolrConfig;
 import org.apache.solr.core.SolrCore;
-import org.apache.solr.core.SolrInfoMBean;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.core.SolrXmlConfig;
 import org.apache.solr.handler.UpdateRequestHandler;


[06/44] lucene-solr:jira/solr-9854: SOLR-1953: It may be possible for temporary files to accumulate until the Solr process is shut down.

Posted by ab...@apache.org.
SOLR-1953: It may be possible for temporary files to accumulate until the Solr process is shut down.


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

Branch: refs/heads/jira/solr-9854
Commit: e82399d0677651ad4be1d8d2bdc4777b5d90b0fa
Parents: 22d9af4
Author: markrmiller <ma...@apache.org>
Authored: Mon Dec 12 11:10:58 2016 -0500
Committer: markrmiller <ma...@apache.org>
Committed: Wed Dec 14 11:16:50 2016 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../apache/solr/servlet/SolrDispatchFilter.java |  15 ++
 .../apache/solr/servlet/SolrRequestParsers.java |  31 ++--
 .../solr/util/SolrFileCleaningTracker.java      | 147 +++++++++++++++++++
 4 files changed, 182 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e82399d0/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 41af0ff..946a04e 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -244,6 +244,9 @@ Bug Fixes
 
 * SOLR-9823: CoreContainer incorrectly setting MDCLoggingContext for core (Jessica Cheng Mallet via Erick Erickson)
 
+* SOLR-1953: It may be possible for temporary files to accumulate until the Solr process is shut down.
+  (Karl Wright, Mark Miller)
+
 Other Changes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e82399d0/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
index 5a4cfb6..e8c4657 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
@@ -45,6 +45,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import org.apache.commons.io.FileCleaningTracker;
 import org.apache.commons.io.input.CloseShieldInputStream;
 import org.apache.commons.io.output.CloseShieldOutputStream;
 import org.apache.commons.lang.StringUtils;
@@ -62,6 +63,7 @@ import org.apache.solr.core.SolrXmlConfig;
 import org.apache.solr.request.SolrRequestInfo;
 import org.apache.solr.security.AuthenticationPlugin;
 import org.apache.solr.security.PKIAuthenticationPlugin;
+import org.apache.solr.util.SolrFileCleaningTracker;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -123,6 +125,8 @@ public class SolrDispatchFilter extends BaseSolrFilter {
   {
     log.trace("SolrDispatchFilter.init(): {}", this.getClass().getClassLoader());
 
+    SolrRequestParsers.fileCleaningTracker = new SolrFileCleaningTracker();
+    
     StartupLoggingUtils.checkLogDir();
     logWelcomeBanner();
     String muteConsole = System.getProperty(SOLR_LOG_MUTECONSOLE);
@@ -240,6 +244,17 @@ public class SolrDispatchFilter extends BaseSolrFilter {
   
   @Override
   public void destroy() {
+    try {
+      FileCleaningTracker fileCleaningTracker = SolrRequestParsers.fileCleaningTracker;
+      if (fileCleaningTracker != null) {
+        fileCleaningTracker.exitWhenFinished();
+      }
+    } catch (Exception e) {
+      log.warn("Exception closing FileCleaningTracker", e);
+    } finally {
+      SolrRequestParsers.fileCleaningTracker = null;
+    }
+
     if (cores != null) {
       try {
         cores.shutdown();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e82399d0/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java b/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java
index 9d7e7d9..968320e 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java
@@ -42,6 +42,7 @@ import java.util.Map;
 import org.apache.commons.fileupload.FileItem;
 import org.apache.commons.fileupload.disk.DiskFileItemFactory;
 import org.apache.commons.fileupload.servlet.ServletFileUpload;
+import org.apache.commons.io.FileCleaningTracker;
 import org.apache.commons.io.input.CloseShieldInputStream;
 import org.apache.lucene.util.IOUtils;
 import org.apache.solr.common.SolrException;
@@ -58,6 +59,7 @@ import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequestBase;
 import org.apache.solr.util.RTimerTree;
+import org.apache.solr.util.SolrFileCleaningTracker;
 
 import static org.apache.solr.common.params.CommonParams.PATH;
 
@@ -88,6 +90,8 @@ public class SolrRequestParsers
   /** Default instance for e.g. admin requests. Limits to 2 MB uploads and does not allow remote streams. */
   public static final SolrRequestParsers DEFAULT = new SolrRequestParsers();
   
+  public static volatile SolrFileCleaningTracker fileCleaningTracker;
+  
   /**
    * Pass in an xml configuration.  A null configuration will enable
    * everything with maximum values.
@@ -532,31 +536,30 @@ public class SolrRequestParsers
   /**
    * Extract Multipart streams
    */
-  static class MultipartRequestParser implements SolrRequestParser
-  {
+  static class MultipartRequestParser implements SolrRequestParser {
     private final int uploadLimitKB;
+    private DiskFileItemFactory factory = new DiskFileItemFactory();
     
-    public MultipartRequestParser( int limit )
-    {
+    public MultipartRequestParser(int limit) {
       uploadLimitKB = limit;
+
+      // Set factory constraints
+      FileCleaningTracker fct = fileCleaningTracker;
+      if (fct != null) {
+        factory.setFileCleaningTracker(fileCleaningTracker);
+      }
+      // TODO - configure factory.setSizeThreshold(yourMaxMemorySize);
+      // TODO - configure factory.setRepository(yourTempDirectory);
     }
     
     @Override
-    public SolrParams parseParamsAndFillStreams( 
-        final HttpServletRequest req, ArrayList<ContentStream> streams ) throws Exception
-    {
+    public SolrParams parseParamsAndFillStreams(
+        final HttpServletRequest req, ArrayList<ContentStream> streams) throws Exception {
       if( !ServletFileUpload.isMultipartContent(req) ) {
         throw new SolrException( ErrorCode.BAD_REQUEST, "Not multipart content! "+req.getContentType() );
       }
       
       MultiMapSolrParams params = parseQueryString( req.getQueryString() );
-      
-      // Create a factory for disk-based file items
-      DiskFileItemFactory factory = new DiskFileItemFactory();
-
-      // Set factory constraints
-      // TODO - configure factory.setSizeThreshold(yourMaxMemorySize);
-      // TODO - configure factory.setRepository(yourTempDirectory);
 
       // Create a new file upload handler
       ServletFileUpload upload = new ServletFileUpload(factory);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e82399d0/solr/core/src/java/org/apache/solr/util/SolrFileCleaningTracker.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/SolrFileCleaningTracker.java b/solr/core/src/java/org/apache/solr/util/SolrFileCleaningTracker.java
new file mode 100644
index 0000000..9c66f0f
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/util/SolrFileCleaningTracker.java
@@ -0,0 +1,147 @@
+/*
+ * 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.File;
+import java.lang.ref.PhantomReference;
+import java.lang.ref.ReferenceQueue;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+
+import org.apache.commons.io.FileCleaningTracker;
+import org.apache.commons.io.FileDeleteStrategy;
+
+public class SolrFileCleaningTracker extends FileCleaningTracker {
+
+  ReferenceQueue<Object> q = new ReferenceQueue<>();
+
+  final Collection<Tracker> trackers = Collections.synchronizedSet(new HashSet<Tracker>());
+
+  final List<String> deleteFailures = Collections.synchronizedList(new ArrayList<String>());
+
+  volatile boolean exitWhenFinished = false;
+
+  Thread reaper;
+
+  public void track(final File file, final Object marker) {
+    track(file, marker, null);
+  }
+
+  public void track(final File file, final Object marker, final FileDeleteStrategy deleteStrategy) {
+    if (file == null) {
+      throw new NullPointerException("The file must not be null");
+    }
+    addTracker(file.getPath(), marker, deleteStrategy);
+  }
+
+  public void track(final String path, final Object marker) {
+    track(path, marker, null);
+  }
+
+  public void track(final String path, final Object marker, final FileDeleteStrategy deleteStrategy) {
+    if (path == null) {
+      throw new NullPointerException("The path must not be null");
+    }
+    addTracker(path, marker, deleteStrategy);
+  }
+
+  private synchronized void addTracker(final String path, final Object marker,
+      final FileDeleteStrategy deleteStrategy) {
+    if (exitWhenFinished) {
+      throw new IllegalStateException("No new trackers can be added once exitWhenFinished() is called");
+    }
+    if (reaper == null) {
+      reaper = new Reaper();
+      reaper.start();
+    }
+    trackers.add(new Tracker(path, deleteStrategy, marker, q));
+  }
+
+  public int getTrackCount() {
+    return trackers.size();
+  }
+
+  public List<String> getDeleteFailures() {
+    return deleteFailures;
+  }
+
+  public synchronized void exitWhenFinished() {
+    // synchronized block protects reaper
+    exitWhenFinished = true;
+    if (reaper != null) {
+      synchronized (reaper) {
+        reaper.interrupt();
+        try {
+          reaper.join();
+        } catch (InterruptedException e) { 
+          Thread.currentThread().interrupt();
+        }
+      }
+    }
+  }
+
+  private final class Reaper extends Thread {
+    Reaper() {
+      super("MultiPart Upload Tmp File Reaper");
+      setDaemon(true);
+    }
+
+    @Override
+    public void run() {
+      while (exitWhenFinished == false || trackers.size() > 0) {
+        try {
+          // Wait for a tracker to remove.
+          final Tracker tracker = (Tracker) q.remove(); // cannot return null
+          trackers.remove(tracker);
+          if (!tracker.delete()) {
+            deleteFailures.add(tracker.getPath());
+          }
+          tracker.clear();
+        } catch (final InterruptedException e) {
+          Thread.currentThread().interrupt();
+          break;
+        }
+      }
+    }
+  }
+
+  private static final class Tracker extends PhantomReference<Object> {
+
+    private final String path;
+
+    private final FileDeleteStrategy deleteStrategy;
+
+    Tracker(final String path, final FileDeleteStrategy deleteStrategy, final Object marker,
+        final ReferenceQueue<? super Object> queue) {
+      super(marker, queue);
+      this.path = path;
+      this.deleteStrategy = deleteStrategy == null ? FileDeleteStrategy.NORMAL : deleteStrategy;
+    }
+
+    public String getPath() {
+      return path;
+    }
+
+    public boolean delete() {
+      return deleteStrategy.deleteQuietly(new File(path));
+    }
+  }
+
+}
\ No newline at end of file


[22/44] lucene-solr:jira/solr-9854: SOLR-9873: tests - fix SolrTestCaseJ4.compareSolrDocument num fields comparison

Posted by ab...@apache.org.
SOLR-9873: tests - fix SolrTestCaseJ4.compareSolrDocument num fields comparison


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

Branch: refs/heads/jira/solr-9854
Commit: dcf202a95813d72b1fd56daa7e30cbf413b891b9
Parents: 5f4521d
Author: yonik <yo...@apache.org>
Authored: Fri Dec 16 18:41:02 2016 -0500
Committer: yonik <yo...@apache.org>
Committed: Fri Dec 16 18:41:02 2016 -0500

----------------------------------------------------------------------
 solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dcf202a9/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
index 19bf601..437679d 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
@@ -1948,7 +1948,7 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
     SolrDocument solrDocument1 = (SolrDocument) expected;
     SolrDocument solrDocument2 = (SolrDocument) actual;
 
-    if(solrDocument1.getFieldNames().size() != solrDocument1.getFieldNames().size()) {
+    if(solrDocument1.getFieldNames().size() != solrDocument2.getFieldNames().size()) {
       return false;
     }
 


[44/44] lucene-solr:jira/solr-9854: Merge branch 'master' into jira/solr-9854

Posted by ab...@apache.org.
Merge branch 'master' into jira/solr-9854


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

Branch: refs/heads/jira/solr-9854
Commit: fa87f6a000e99f29d6cbb9c0bad8355cc1f353e5
Parents: 3562ef7 8bbdb62
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Tue Dec 20 11:44:43 2016 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Tue Dec 20 11:44:43 2016 +0100

----------------------------------------------------------------------
 .../dot.settings/org.eclipse.jdt.core.prefs     |    1 +
 .../search/TestDocValuesStatsCollector.java     |   29 +-
 lucene/tools/javadoc/ecj.javadocs.prefs         |    2 +-
 solr/CHANGES.txt                                |    7 +
 solr/contrib/ltr/README.md                      |   36 +-
 solr/contrib/ltr/example/solrconfig.xml         | 1711 ------------------
 .../org/apache/solr/ltr/CSVFeatureLogger.java   |   62 +
 .../java/org/apache/solr/ltr/FeatureLogger.java |  134 +-
 .../java/org/apache/solr/ltr/LTRRescorer.java   |    2 +-
 .../org/apache/solr/ltr/LTRScoringQuery.java    |    2 +-
 .../solr/ltr/SolrQueryRequestContextUtils.java  |    6 +-
 .../org/apache/solr/ltr/model/LinearModel.java  |   10 +
 .../ltr/model/MultipleAdditiveTreesModel.java   |    5 +
 .../LTRFeatureLoggerTransformerFactory.java     |  105 +-
 solr/contrib/ltr/src/java/overview.html         |    2 +-
 .../solr/collection1/conf/solrconfig-ltr.xml    |    5 +-
 .../collection1/conf/solrconfig-ltr_Th10_10.xml |    5 +-
 .../collection1/conf/solrconfig-multiseg.xml    |   12 +-
 .../apache/solr/ltr/FeatureLoggerTestUtils.java |   44 +
 .../org/apache/solr/ltr/TestLTROnSolrCloud.java |   17 +-
 .../solr/ltr/TestSelectiveWeightCreation.java   |   11 +-
 .../solr/ltr/feature/TestExternalFeatures.java  |   12 +-
 .../ltr/feature/TestExternalValueFeatures.java  |    4 +-
 .../solr/ltr/feature/TestFeatureLogging.java    |   99 +-
 .../solr/ltr/feature/TestFieldValueFeature.java |   13 +-
 .../solr/ltr/feature/TestFilterSolrFeature.java |    6 +-
 .../ltr/feature/TestNoMatchSolrFeature.java     |    6 +-
 .../ltr/feature/TestOriginalScoreFeature.java   |    9 +-
 .../solr/SolrMorphlineZkAliasTest.java          |    7 -
 .../org/apache/solr/cloud/CreateAliasCmd.java   |   21 +-
 .../src/java/org/apache/solr/core/SolrCore.java |    4 +-
 .../solr/handler/admin/CollectionsHandler.java  |    2 +-
 .../security/AttributeOnlyServletContext.java   |  291 +++
 .../ConfigurableInternodeAuthHadoopPlugin.java  |   68 +
 .../security/DelegationTokenKerberosFilter.java |    6 +-
 .../solr/security/GenericHadoopAuthPlugin.java  |  266 +++
 .../apache/solr/security/HadoopAuthFilter.java  |  198 ++
 .../apache/solr/security/HadoopAuthPlugin.java  |  241 +++
 .../apache/solr/security/KerberosFilter.java    |    6 +-
 .../apache/solr/security/KerberosPlugin.java    |  314 +---
 ...tContinuesRecorderAuthenticationHandler.java |   71 +
 .../solr/security/hadoop_kerberos_config.json   |   16 +
 .../hadoop_simple_auth_with_delegation.json     |   29 +
 .../apache/solr/cloud/AliasIntegrationTest.java |   50 +-
 .../TestSolrCloudWithSecureImpersonation.java   |    8 +-
 .../solr/security/hadoop/ImpersonationUtil.java |   73 +
 .../hadoop/ImpersonatorCollectionsHandler.java  |   60 +
 .../hadoop/TestDelegationWithHadoopAuth.java    |  400 ++++
 .../hadoop/TestImpersonationWithHadoopAuth.java |  215 +++
 .../TestSolrCloudWithHadoopAuthPlugin.java      |  136 ++
 .../conf/solrconfig.xml                         |   54 +-
 .../impl/DelegationTokenHttpSolrClient.java     |   34 +-
 .../solrj/impl/HttpClientBuilderFactory.java    |   41 +
 .../solr/client/solrj/impl/HttpSolrClient.java  |   56 +-
 .../solrj/impl/Krb5HttpClientBuilder.java       |   10 +-
 .../solrj/impl/BasicHttpSolrClientTest.java     |   31 +
 .../solr/client/solrj/io/sql/JdbcTest.java      |    7 +-
 .../client/solrj/io/stream/JDBCStreamTest.java  |    9 +-
 .../solrj/io/stream/StreamExpressionTest.java   |    7 +-
 .../client/solrj/io/stream/StreamingTest.java   |   10 +-
 .../apache/solr/cloud/MiniSolrCloudCluster.java |   29 +-
 .../apache/solr/cloud/SolrCloudTestCase.java    |   30 +-
 62 files changed, 2781 insertions(+), 2376 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fa87f6a0/solr/CHANGES.txt
----------------------------------------------------------------------
diff --cc solr/CHANGES.txt
index 34494b1,519cdfa..52112cd
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@@ -173,6 -173,12 +173,11 @@@ New Feature
  * SOLR-9844: FieldCache information fetched via the mbeans handler or seen via the UI now displays the total size used.
    The individual cache entries in the response are now formatted better as well. (Varun Thacker)
  
 -<<<<<<< HEAD
+ * SOLR-9513: Generic authentication plugins (GenericHadoopAuthPlugin and ConfigurableInternodeAuthHadoopPlugin) that delegate
+   all functionality to Hadoop authentication framework. (Hrishikesh Gadre via Ishan Chattopadhyaya)
+ 
+ * SOLR-9860: Enable configuring invariantParams via HttpSolrClient.Builder (Hrishikesh Gadre, Ishan Chattopadhyaya)
+  
  * SOLR-4735: Improve metrics reporting. This uses the dropwizard metric library, adding an internal API
    for registering and reporting metrics from Solr components. Several new metrics and an improved JMX
    reporter have been added (Alan Woodward, Jeff Wartes, Christine Poerschke, Kelvin Wong, shalin, ab)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fa87f6a0/solr/core/src/java/org/apache/solr/core/SolrCore.java
----------------------------------------------------------------------


[11/44] lucene-solr:jira/solr-9854: LUCENE-7590: add sum, variance and stdev stats to NumericDVStats

Posted by ab...@apache.org.
LUCENE-7590: add sum, variance and stdev stats to NumericDVStats


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

Branch: refs/heads/jira/solr-9854
Commit: 295cab7216ca76debaf4d354409741058a8641a1
Parents: e4f31fa
Author: Shai Erera <sh...@apache.org>
Authored: Thu Dec 15 12:52:37 2016 +0200
Committer: Shai Erera <sh...@apache.org>
Committed: Thu Dec 15 14:42:08 2016 +0200

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  3 +
 .../apache/lucene/search/DocValuesStats.java    | 39 +++++++++++-
 .../search/TestDocValuesStatsCollector.java     | 62 +++++++++++++++++---
 3 files changed, 95 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/295cab72/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index f38c0d5..0e327d2 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -67,6 +67,9 @@ New features
 
 * LUCENE-7466: Added AxiomaticSimilarity. (Peilin Yang via Tommaso Teofili)
 
+* LUCENE-7590: Added DocValuesStatsCollector to compute statistics on DocValues
+  fields. (Shai Erera)
+
 Bug Fixes
 
 * LUCENE-7547: JapaneseTokenizerFactory was failing to close the

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/295cab72/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java b/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
index 998bef4..c8b7752 100644
--- a/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
+++ b/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
@@ -98,6 +98,7 @@ public abstract class DocValuesStats<T> {
   public static abstract class NumericDocValuesStats<T extends Number> extends DocValuesStats<T> {
 
     protected double mean = 0.0;
+    protected double variance = 0.0;
 
     protected NumericDocValues ndv;
 
@@ -116,15 +117,32 @@ public abstract class DocValuesStats<T> {
       return ndv.advanceExact(doc);
     }
 
-    /** The mean of all values of the field. Undefined when {@link #count} is zero. */
+    /** The mean of all values of the field. */
     public final double mean() {
       return mean;
     }
+
+    /** Returns the variance of all values of the field. */
+    public final double variance() {
+      int count = count();
+      return count > 0 ? variance / count : 0;
+    }
+
+    /** Returns the stdev of all values of the field. */
+    public final double stdev() {
+      return Math.sqrt(variance());
+    }
+
+    /** Returns the sum of values of the field. Note that if the values are large, the {@code sum} might overflow. */
+    public abstract T sum();
   }
 
   /** Holds DocValues statistics for a numeric field storing {@code long} values. */
   public static final class LongDocValuesStats extends NumericDocValuesStats<Long> {
 
+    // To avoid boxing 'long' to 'Long' while the sum is computed, declare it as private variable.
+    private long sum = 0;
+
     public LongDocValuesStats(String field) {
       super(field, Long.MAX_VALUE, Long.MIN_VALUE);
     }
@@ -138,13 +156,24 @@ public abstract class DocValuesStats<T> {
       if (val < min) {
         min = val;
       }
+      sum += val;
+      double oldMean = mean;
       mean += (val - mean) / count;
+      variance += (val - mean) * (val - oldMean);
+    }
+
+    @Override
+    public Long sum() {
+      return sum;
     }
   }
 
   /** Holds DocValues statistics for a numeric field storing {@code double} values. */
   public static final class DoubleDocValuesStats extends NumericDocValuesStats<Double> {
 
+    // To avoid boxing 'double' to 'Double' while the sum is computed, declare it as private variable.
+    private double sum = 0;
+
     public DoubleDocValuesStats(String field) {
       super(field, Double.MAX_VALUE, Double.MIN_VALUE);
     }
@@ -158,7 +187,15 @@ public abstract class DocValuesStats<T> {
       if (Double.compare(val, min) < 0) {
         min = val;
       }
+      sum += val;
+      double oldMean = mean;
       mean += (val - mean) / count;
+      variance += (val - mean) * (val - oldMean);
+    }
+
+    @Override
+    public Double sum() {
+      return sum;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/295cab72/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java b/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
index 65f82e6..8f8b09e 100644
--- a/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
+++ b/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
@@ -18,6 +18,8 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.DoubleSummaryStatistics;
+import java.util.LongSummaryStatistics;
 import java.util.stream.DoubleStream;
 import java.util.stream.LongStream;
 
@@ -57,7 +59,33 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
     }
   }
 
-  public void testRandomDocsWithLongValues() throws IOException {
+  public void testOneDoc() throws IOException {
+    try (Directory dir = newDirectory();
+        IndexWriter indexWriter = new IndexWriter(dir, newIndexWriterConfig())) {
+      String field = "numeric";
+      Document doc = new Document();
+      doc.add(new NumericDocValuesField(field, 1));
+      doc.add(new StringField("id", "doc1", Store.NO));
+      indexWriter.addDocument(doc);
+
+      try (DirectoryReader reader = DirectoryReader.open(indexWriter)) {
+        IndexSearcher searcher = new IndexSearcher(reader);
+        LongDocValuesStats stats = new LongDocValuesStats(field);
+        searcher.search(new MatchAllDocsQuery(), new DocValuesStatsCollector(stats));
+
+        assertEquals(1, stats.count());
+        assertEquals(0, stats.missing());
+        assertEquals(1, stats.max().longValue());
+        assertEquals(1, stats.min().longValue());
+        assertEquals(1, stats.sum().longValue());
+        assertEquals(1, stats.mean(), 0.0001);
+        assertEquals(0, stats.variance(), 0.0001);
+        assertEquals(0, stats.stdev(), 0.0001);
+      }
+    }
+  }
+
+  public void testDocsWithLongValues() throws IOException {
     try (Directory dir = newDirectory();
         IndexWriter indexWriter = new IndexWriter(dir, newIndexWriterConfig())) {
       String field = "numeric";
@@ -94,15 +122,20 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
         assertEquals(expCount, stats.count());
         assertEquals(getZeroValues(docValues).count() - reader.numDeletedDocs(), stats.missing());
         if (stats.count() > 0) {
-          assertEquals(getPositiveValues(docValues).max().getAsLong(), stats.max().longValue());
-          assertEquals(getPositiveValues(docValues).min().getAsLong(), stats.min().longValue());
-          assertEquals(getPositiveValues(docValues).average().getAsDouble(), stats.mean(), 0.00001);
+          LongSummaryStatistics sumStats = getPositiveValues(docValues).summaryStatistics();
+          assertEquals(sumStats.getMax(), stats.max().longValue());
+          assertEquals(sumStats.getMin(), stats.min().longValue());
+          assertEquals(sumStats.getAverage(), stats.mean(), 0.00001);
+          assertEquals(sumStats.getSum(), stats.sum().longValue());
+          double variance = computeVariance(docValues, stats.mean, stats.count());
+          assertEquals(variance, stats.variance(), 0.00001);
+          assertEquals(Math.sqrt(variance), stats.stdev(), 0.00001);
         }
       }
     }
   }
 
-  public void testRandomDocsWithDoubleValues() throws IOException {
+  public void testDocsWithDoubleValues() throws IOException {
     try (Directory dir = newDirectory();
         IndexWriter indexWriter = new IndexWriter(dir, newIndexWriterConfig())) {
       String field = "numeric";
@@ -139,9 +172,14 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
         assertEquals(expCount, stats.count());
         assertEquals(getZeroValues(docValues).count() - reader.numDeletedDocs(), stats.missing());
         if (stats.count() > 0) {
-          assertEquals(getPositiveValues(docValues).max().getAsDouble(), stats.max().doubleValue(), 0.00001);
-          assertEquals(getPositiveValues(docValues).min().getAsDouble(), stats.min().doubleValue(), 0.00001);
-          assertEquals(getPositiveValues(docValues).average().getAsDouble(), stats.mean(), 0.00001);
+          DoubleSummaryStatistics sumStats = getPositiveValues(docValues).summaryStatistics();
+          assertEquals(sumStats.getMax(), stats.max().doubleValue(), 0.00001);
+          assertEquals(sumStats.getMin(), stats.min().doubleValue(), 0.00001);
+          assertEquals(sumStats.getAverage(), stats.mean(), 0.00001);
+          assertEquals(sumStats.getSum(), stats.sum(), 0.00001);
+          double variance = computeVariance(docValues, stats.mean, stats.count());
+          assertEquals(variance, stats.variance(), 0.00001);
+          assertEquals(Math.sqrt(variance), stats.stdev(), 0.00001);
         }
       }
     }
@@ -163,4 +201,12 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
     return Arrays.stream(docValues).filter(v -> v == 0);
   }
 
+  private static double computeVariance(long[] values, double mean, int count) {
+    return getPositiveValues(values).mapToDouble(v -> (v - mean) * (v-mean)).sum() / count;
+  }
+
+  private static double computeVariance(double[] values, double mean, int count) {
+    return getPositiveValues(values).map(v -> (v - mean) * (v-mean)).sum() / count;
+  }
+
 }