You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sh...@apache.org on 2017/03/14 06:40:38 UTC

[01/43] lucene-solr:feature/autoscaling: SOLR-8593: in TestSQLHandler assume not run with Turkish locale

Repository: lucene-solr
Updated Branches:
  refs/heads/feature/autoscaling acb185b2d -> faeb1fe8c


SOLR-8593: in TestSQLHandler assume not run with Turkish locale


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

Branch: refs/heads/feature/autoscaling
Commit: 6df17c8cfe72d229140fb644d067a50cd7a2b455
Parents: acb185b
Author: Joel Bernstein <jb...@apache.org>
Authored: Mon Mar 6 12:20:28 2017 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Mon Mar 6 12:20:28 2017 -0500

----------------------------------------------------------------------
 solr/core/src/test/org/apache/solr/handler/TestSQLHandler.java | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6df17c8c/solr/core/src/test/org/apache/solr/handler/TestSQLHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/TestSQLHandler.java b/solr/core/src/test/org/apache/solr/handler/TestSQLHandler.java
index d724fbd..f222cee 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestSQLHandler.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestSQLHandler.java
@@ -19,6 +19,7 @@ package org.apache.solr.handler;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Locale;
 
 import org.apache.solr.client.solrj.io.Tuple;
 import org.apache.solr.client.solrj.io.stream.ExceptionStream;
@@ -69,6 +70,9 @@ public class TestSQLHandler extends AbstractFullDistribZkTestBase {
 
   @Test
   public void doTest() throws Exception {
+
+    assumeFalse("This test fails on UNIX with Turkish default locale", new Locale("tr").getLanguage().equals(Locale.getDefault().getLanguage()));
+
     waitForRecoveriesToFinish(false);
 
     testBasicSelect();


[35/43] lucene-solr:feature/autoscaling: fix IntRange.newIntersectsQuery to use newRelationQuery helper

Posted by sh...@apache.org.
fix IntRange.newIntersectsQuery to use newRelationQuery helper


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

Branch: refs/heads/feature/autoscaling
Commit: 35e0c05dbb098fb14e04bf8e48efa1a7943f980a
Parents: d34d81f
Author: Nicholas Knize <nk...@gmail.com>
Authored: Mon Mar 13 02:36:57 2017 -0500
Committer: Nicholas Knize <nk...@gmail.com>
Committed: Mon Mar 13 02:36:57 2017 -0500

----------------------------------------------------------------------
 lucene/core/src/java/org/apache/lucene/document/IntRange.java | 7 +------
 1 file changed, 1 insertion(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/35e0c05d/lucene/core/src/java/org/apache/lucene/document/IntRange.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/IntRange.java b/lucene/core/src/java/org/apache/lucene/document/IntRange.java
index 2618f14..6d2b71c 100644
--- a/lucene/core/src/java/org/apache/lucene/document/IntRange.java
+++ b/lucene/core/src/java/org/apache/lucene/document/IntRange.java
@@ -188,12 +188,7 @@ public class IntRange extends Field {
    * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
    */
   public static Query newIntersectsQuery(String field, final int[] min, final int[] max) {
-    return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.INTERSECTS) {
-      @Override
-      protected String toString(byte[] ranges, int dimension) {
-        return IntRange.toString(ranges, dimension);
-      }
-    };
+    return newRelationQuery(field, min, max, QueryType.INTERSECTS);
   }
 
   /**


[04/43] lucene-solr:feature/autoscaling: SOLR-9986: Fix precommit

Posted by sh...@apache.org.
SOLR-9986: Fix precommit


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

Branch: refs/heads/feature/autoscaling
Commit: 7af6cc97151d727cb4d4e8730491a32e56a29397
Parents: 3131ec2
Author: Cao Manh Dat <da...@apache.org>
Authored: Tue Mar 7 15:37:24 2017 +0700
Committer: Cao Manh Dat <da...@apache.org>
Committed: Tue Mar 7 15:37:24 2017 +0700

----------------------------------------------------------------------
 solr/core/src/java/org/apache/solr/request/NumericFacets.java       | 1 -
 solr/core/src/java/org/apache/solr/schema/DatePointField.java       | 1 -
 .../solr/update/processor/ParsingFieldUpdateProcessorsTest.java     | 1 -
 3 files changed, 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7af6cc97/solr/core/src/java/org/apache/solr/request/NumericFacets.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/request/NumericFacets.java b/solr/core/src/java/org/apache/solr/request/NumericFacets.java
index c3bcb9f..fd17f1f 100644
--- a/solr/core/src/java/org/apache/solr/request/NumericFacets.java
+++ b/solr/core/src/java/org/apache/solr/request/NumericFacets.java
@@ -52,7 +52,6 @@ import org.apache.solr.schema.TrieField;
 import org.apache.solr.search.DocIterator;
 import org.apache.solr.search.DocSet;
 import org.apache.solr.search.SolrIndexSearcher;
-import org.apache.solr.util.DateMathParser;
 
 /** Utility class to compute facets on numeric fields. */
 final class NumericFacets {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7af6cc97/solr/core/src/java/org/apache/solr/schema/DatePointField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/DatePointField.java b/solr/core/src/java/org/apache/solr/schema/DatePointField.java
index 18bf651..b3517db 100644
--- a/solr/core/src/java/org/apache/solr/schema/DatePointField.java
+++ b/solr/core/src/java/org/apache/solr/schema/DatePointField.java
@@ -25,7 +25,6 @@ import java.util.Date;
 import org.apache.lucene.document.LongPoint;
 import org.apache.lucene.document.StoredField;
 import org.apache.lucene.index.IndexableField;
-import org.apache.lucene.legacy.LegacyNumericRangeQuery;
 import org.apache.lucene.legacy.LegacyNumericType;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.valuesource.LongFieldSource;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7af6cc97/solr/core/src/test/org/apache/solr/update/processor/ParsingFieldUpdateProcessorsTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/processor/ParsingFieldUpdateProcessorsTest.java b/solr/core/src/test/org/apache/solr/update/processor/ParsingFieldUpdateProcessorsTest.java
index 31f4760..3aeb1fb 100644
--- a/solr/core/src/test/org/apache/solr/update/processor/ParsingFieldUpdateProcessorsTest.java
+++ b/solr/core/src/test/org/apache/solr/update/processor/ParsingFieldUpdateProcessorsTest.java
@@ -18,7 +18,6 @@ package org.apache.solr.update.processor;
 
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.schema.IndexSchema;
-import org.apache.solr.schema.PointField;
 import org.joda.time.DateTime;
 import org.joda.time.DateTimeZone;
 import org.joda.time.format.DateTimeFormat;


[40/43] lucene-solr:feature/autoscaling: SOLR-9045: Make RecoveryStrategy settings configurable.

Posted by sh...@apache.org.
SOLR-9045: Make RecoveryStrategy settings configurable.


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

Branch: refs/heads/feature/autoscaling
Commit: c8bad8c10ac52d89318932636b1e1401c314b5e4
Parents: ceffbf9
Author: Christine Poerschke <cp...@apache.org>
Authored: Mon Mar 13 15:49:01 2017 +0000
Committer: Christine Poerschke <cp...@apache.org>
Committed: Mon Mar 13 15:49:01 2017 +0000

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   2 +
 .../org/apache/solr/cloud/RecoveryStrategy.java | 106 ++++++++++++++----
 .../java/org/apache/solr/core/SolrConfig.java   |   2 +
 .../src/java/org/apache/solr/core/SolrCore.java |  22 +++-
 .../solr/update/DefaultSolrCoreState.java       |  15 ++-
 .../org/apache/solr/update/SolrCoreState.java   |   6 +
 .../solrconfig-configurerecoverystrategy.xml    |  28 +++++
 .../conf/solrconfig-customrecoverystrategy.xml  |  32 ++++++
 .../core/ConfigureRecoveryStrategyTest.java     | 111 +++++++++++++++++++
 9 files changed, 300 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c8bad8c1/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 5d6d9d7..1469d3e 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -173,6 +173,8 @@ New Features
   However, if there is a leader election while this request is in transit, the versions may not be returned from that
   shard. (Boris Naguet, Ishan Chattopadhyaya)
 
+* SOLR-9045: Make RecoveryStrategy settings configurable. (Christine Poerschke)
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c8bad8c1/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
index 3bd2e74..8865c08 100644
--- a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
+++ b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
@@ -46,6 +46,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.cloud.ZooKeeperException;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.UpdateParams;
+import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.core.DirectoryFactory.DirContext;
@@ -62,17 +63,43 @@ import org.apache.solr.update.UpdateLog;
 import org.apache.solr.update.UpdateLog.RecoveryInfo;
 import org.apache.solr.update.processor.DistributedUpdateProcessor;
 import org.apache.solr.util.RefCounted;
+import org.apache.solr.util.SolrPluginUtils;
+import org.apache.solr.util.plugin.NamedListInitializedPlugin;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+/**
+ * This class may change in future and customisations are not supported
+ * between versions in terms of API or back compat behaviour.
+ * @lucene.experimental
+ */
 public class RecoveryStrategy extends Thread implements Closeable {
 
+  public static class Builder implements NamedListInitializedPlugin {
+    private NamedList args;
+    @Override
+    public void init(NamedList args) {
+      this.args = args;
+    }
+    // this should only be used from SolrCoreState
+    public RecoveryStrategy create(CoreContainer cc, CoreDescriptor cd,
+        RecoveryStrategy.RecoveryListener recoveryListener) {
+      final RecoveryStrategy recoveryStrategy = newRecoveryStrategy(cc, cd, recoveryListener);
+      SolrPluginUtils.invokeSetters(recoveryStrategy, args);
+      return recoveryStrategy;
+    }
+    protected RecoveryStrategy newRecoveryStrategy(CoreContainer cc, CoreDescriptor cd,
+        RecoveryStrategy.RecoveryListener recoveryListener) {
+      return new RecoveryStrategy(cc, cd, recoveryListener);
+    }
+  }
+
   private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  private static final int WAIT_FOR_UPDATES_WITH_STALE_STATE_PAUSE = Integer.getInteger("solr.cloud.wait-for-updates-with-stale-state-pause", 2500);
-  private static final int MAX_RETRIES = 500;
-  private static final int STARTING_RECOVERY_DELAY = 5000;
+  private int waitForUpdatesWithStaleStatePauseMilliSeconds = Integer.getInteger("solr.cloud.wait-for-updates-with-stale-state-pause", 2500);
+  private int maxRetries = 500;
+  private int startingRecoveryDelayMilliSeconds = 5000;
 
   public static interface RecoveryListener {
     public void recovered();
@@ -92,8 +119,7 @@ public class RecoveryStrategy extends Thread implements Closeable {
   private CoreContainer cc;
   private volatile HttpUriRequest prevSendPreRecoveryHttpUriRequest;
   
-  // this should only be used from SolrCoreState
-  public RecoveryStrategy(CoreContainer cc, CoreDescriptor cd, RecoveryListener recoveryListener) {
+  protected RecoveryStrategy(CoreContainer cc, CoreDescriptor cd, RecoveryListener recoveryListener) {
     this.cc = cc;
     this.coreName = cd.getName();
     this.recoveryListener = recoveryListener;
@@ -104,13 +130,41 @@ public class RecoveryStrategy extends Thread implements Closeable {
     coreZkNodeName = cd.getCloudDescriptor().getCoreNodeName();
   }
 
-  public void setRecoveringAfterStartup(boolean recoveringAfterStartup) {
+  final public int getWaitForUpdatesWithStaleStatePauseMilliSeconds() {
+    return waitForUpdatesWithStaleStatePauseMilliSeconds;
+  }
+
+  final public void setWaitForUpdatesWithStaleStatePauseMilliSeconds(int waitForUpdatesWithStaleStatePauseMilliSeconds) {
+    this.waitForUpdatesWithStaleStatePauseMilliSeconds = waitForUpdatesWithStaleStatePauseMilliSeconds;
+  }
+
+  final public int getMaxRetries() {
+    return maxRetries;
+  }
+
+  final public void setMaxRetries(int maxRetries) {
+    this.maxRetries = maxRetries;
+  }
+
+  final public int getStartingRecoveryDelayMilliSeconds() {
+    return startingRecoveryDelayMilliSeconds;
+  }
+
+  final public void setStartingRecoveryDelayMilliSeconds(int startingRecoveryDelayMilliSeconds) {
+    this.startingRecoveryDelayMilliSeconds = startingRecoveryDelayMilliSeconds;
+  }
+
+  final public boolean getRecoveringAfterStartup() {
+    return recoveringAfterStartup;
+  }
+
+  final public void setRecoveringAfterStartup(boolean recoveringAfterStartup) {
     this.recoveringAfterStartup = recoveringAfterStartup;
   }
 
   // make sure any threads stop retrying
   @Override
-  public void close() {
+  final public void close() {
     close = true;
     if (prevSendPreRecoveryHttpUriRequest != null) {
       prevSendPreRecoveryHttpUriRequest.abort();
@@ -118,7 +172,7 @@ public class RecoveryStrategy extends Thread implements Closeable {
     LOG.warn("Stopping recovery for core=[{}] coreNodeName=[{}]", coreName, coreZkNodeName);
   }
 
-  private void recoveryFailed(final SolrCore core,
+  final private void recoveryFailed(final SolrCore core,
       final ZkController zkController, final String baseUrl,
       final String shardZkNodeName, final CoreDescriptor cd) throws KeeperException, InterruptedException {
     SolrException.log(LOG, "Recovery failed - I give up.");
@@ -130,11 +184,19 @@ public class RecoveryStrategy extends Thread implements Closeable {
     }
   }
   
-  private void replicate(String nodeName, SolrCore core, ZkNodeProps leaderprops)
+  /**
+   * This method may change in future and customisations are not supported
+   * between versions in terms of API or back compat behaviour.
+   * @lucene.experimental
+   */
+  protected String getReplicateLeaderUrl(ZkNodeProps leaderprops) {
+    return new ZkCoreNodeProps(leaderprops).getCoreUrl();
+  }
+
+  final private void replicate(String nodeName, SolrCore core, ZkNodeProps leaderprops)
       throws SolrServerException, IOException {
 
-    ZkCoreNodeProps leaderCNodeProps = new ZkCoreNodeProps(leaderprops);
-    String leaderUrl = leaderCNodeProps.getCoreUrl();
+    final String leaderUrl = getReplicateLeaderUrl(leaderprops);
     
     LOG.info("Attempting to replicate from [{}].", leaderUrl);
     
@@ -191,7 +253,7 @@ public class RecoveryStrategy extends Thread implements Closeable {
 
   }
 
-  private void commitOnLeader(String leaderUrl) throws SolrServerException,
+  final private void commitOnLeader(String leaderUrl) throws SolrServerException,
       IOException {
     try (HttpSolrClient client = new HttpSolrClient.Builder(leaderUrl).build()) {
       client.setConnectionTimeout(30000);
@@ -205,7 +267,7 @@ public class RecoveryStrategy extends Thread implements Closeable {
   }
 
   @Override
-  public void run() {
+  final public void run() {
 
     // set request info for logging
     try (SolrCore core = cc.getCore(coreName)) {
@@ -234,7 +296,7 @@ public class RecoveryStrategy extends Thread implements Closeable {
   }
 
   // TODO: perhaps make this grab a new core each time through the loop to handle core reloads?
-  public void doRecovery(SolrCore core) throws KeeperException, InterruptedException {
+  final public void doRecovery(SolrCore core) throws KeeperException, InterruptedException {
     boolean replayed = false;
     boolean successfulRecovery = false;
 
@@ -360,7 +422,7 @@ public class RecoveryStrategy extends Thread implements Closeable {
         // are sure to have finished (see SOLR-7141 for
         // discussion around current value)
         try {
-          Thread.sleep(WAIT_FOR_UPDATES_WITH_STALE_STATE_PAUSE);
+          Thread.sleep(waitForUpdatesWithStaleStatePauseMilliSeconds);
         } catch (InterruptedException e) {
           Thread.currentThread().interrupt();
         }
@@ -479,7 +541,7 @@ public class RecoveryStrategy extends Thread implements Closeable {
           LOG.error("Recovery failed - trying again... (" + retries + ")");
           
           retries++;
-          if (retries >= MAX_RETRIES) {
+          if (retries >= maxRetries) {
             SolrException.log(LOG, "Recovery failed - max retries exceeded (" + retries + ").");
             try {
               recoveryFailed(core, zkController, baseUrl, coreZkNodeName, core.getCoreDescriptor());
@@ -504,7 +566,7 @@ public class RecoveryStrategy extends Thread implements Closeable {
               LOG.info("RecoveryStrategy has been closed");
               break; // check if someone closed us
             }
-            Thread.sleep(STARTING_RECOVERY_DELAY);
+            Thread.sleep(startingRecoveryDelayMilliSeconds);
           }
         } catch (InterruptedException e) {
           Thread.currentThread().interrupt();
@@ -525,7 +587,7 @@ public class RecoveryStrategy extends Thread implements Closeable {
     LOG.info("Finished recovery process, successful=[{}]", Boolean.toString(successfulRecovery));
   }
 
-  private Future<RecoveryInfo> replay(SolrCore core)
+  final private Future<RecoveryInfo> replay(SolrCore core)
       throws InterruptedException, ExecutionException {
     Future<RecoveryInfo> future = core.getUpdateHandler().getUpdateLog().applyBufferedUpdates();
     if (future == null) {
@@ -547,7 +609,7 @@ public class RecoveryStrategy extends Thread implements Closeable {
     return future;
   }
   
-  private void cloudDebugLog(SolrCore core, String op) {
+  final private void cloudDebugLog(SolrCore core, String op) {
     if (!LOG.isDebugEnabled()) {
       return;
     }
@@ -566,11 +628,11 @@ public class RecoveryStrategy extends Thread implements Closeable {
     }
   }
 
-  public boolean isClosed() {
+  final public boolean isClosed() {
     return close;
   }
   
-  private void sendPrepRecoveryCmd(String leaderBaseUrl, String leaderCoreName, Slice slice)
+  final private void sendPrepRecoveryCmd(String leaderBaseUrl, String leaderCoreName, Slice slice)
       throws SolrServerException, IOException, InterruptedException, ExecutionException {
 
     WaitForState prepCmd = new WaitForState();
@@ -603,7 +665,7 @@ public class RecoveryStrategy extends Thread implements Closeable {
     }
   }
 
-  private void sendPrepRecoveryCmd(String leaderBaseUrl, WaitForState prepCmd)
+  final private void sendPrepRecoveryCmd(String leaderBaseUrl, WaitForState prepCmd)
       throws SolrServerException, IOException, InterruptedException, ExecutionException {
     try (HttpSolrClient client = new HttpSolrClient.Builder(leaderBaseUrl).build()) {
       client.setConnectionTimeout(10000);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c8bad8c1/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 bd98075..a244420 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrConfig.java
@@ -58,6 +58,7 @@ import org.apache.lucene.index.IndexDeletionPolicy;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.util.Version;
 import org.apache.solr.client.solrj.io.stream.expr.Expressible;
+import org.apache.solr.cloud.RecoveryStrategy;
 import org.apache.solr.cloud.ZkSolrResourceLoader;
 import org.apache.solr.common.MapSerializable;
 import org.apache.solr.common.SolrException;
@@ -357,6 +358,7 @@ public class SolrConfig extends Config implements MapSerializable {
       .add(new SolrPluginInfo(SolrEventListener.class, "//listener", REQUIRE_CLASS, MULTI_OK, REQUIRE_NAME_IN_OVERLAY))
 
       .add(new SolrPluginInfo(DirectoryFactory.class, "directoryFactory", REQUIRE_CLASS))
+      .add(new SolrPluginInfo(RecoveryStrategy.Builder.class, "recoveryStrategy"))
       .add(new SolrPluginInfo(IndexDeletionPolicy.class, "indexConfig/deletionPolicy", REQUIRE_CLASS))
       .add(new SolrPluginInfo(CodecFactory.class, "codecFactory", REQUIRE_CLASS))
       .add(new SolrPluginInfo(IndexReaderFactory.class, "indexReaderFactory", REQUIRE_CLASS))

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c8bad8c1/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 13c3bdd..70203d4 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -74,6 +74,7 @@ import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.LockObtainFailedException;
 import org.apache.solr.client.solrj.impl.BinaryResponseParser;
 import org.apache.solr.cloud.CloudDescriptor;
+import org.apache.solr.cloud.RecoveryStrategy;
 import org.apache.solr.cloud.ZkSolrResourceLoader;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ClusterState;
@@ -203,6 +204,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
   private final IndexDeletionPolicyWrapper solrDelPolicy;
   private final SolrSnapshotMetaDataManager snapshotMgr;
   private final DirectoryFactory directoryFactory;
+  private final RecoveryStrategy.Builder recoveryStrategyBuilder;
   private IndexReaderFactory indexReaderFactory;
   private final Codec codec;
   private final MemClassLoader memClassLoader;
@@ -657,6 +659,22 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
     return DirectoryFactory.loadDirectoryFactory(solrConfig, getCoreDescriptor().getCoreContainer(), coreMetricManager.getRegistryName());
   }
 
+  private RecoveryStrategy.Builder initRecoveryStrategyBuilder() {
+    final PluginInfo info = solrConfig.getPluginInfo(RecoveryStrategy.Builder.class.getName());
+    final RecoveryStrategy.Builder rsBuilder;
+    if (info != null && info.className != null) {
+      log.info(info.className);
+      rsBuilder = getResourceLoader().newInstance(info.className, RecoveryStrategy.Builder.class);
+    } else {
+      log.info("solr.RecoveryStrategy.Builder");
+      rsBuilder = new RecoveryStrategy.Builder();
+    }
+    if (info != null) {
+      rsBuilder.init(info.initArgs);
+    }
+    return rsBuilder;
+  }
+
   private void initIndexReaderFactory() {
     IndexReaderFactory indexReaderFactory;
     PluginInfo info = solrConfig.getPluginInfo(IndexReaderFactory.class.getName());
@@ -864,10 +882,12 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
 
     if (updateHandler == null) {
       directoryFactory = initDirectoryFactory();
-      solrCoreState = new DefaultSolrCoreState(directoryFactory);
+      recoveryStrategyBuilder = initRecoveryStrategyBuilder();
+      solrCoreState = new DefaultSolrCoreState(directoryFactory, recoveryStrategyBuilder);
     } else {
       solrCoreState = updateHandler.getSolrCoreState();
       directoryFactory = solrCoreState.getDirectoryFactory();
+      recoveryStrategyBuilder = solrCoreState.getRecoveryStrategyBuilder();
       isReloaded = true;
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c8bad8c1/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java b/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java
index 59e35f0..d0daebb 100644
--- a/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java
+++ b/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java
@@ -63,6 +63,7 @@ public final class DefaultSolrCoreState extends SolrCoreState implements Recover
 
   private SolrIndexWriter indexWriter = null;
   private DirectoryFactory directoryFactory;
+  private final RecoveryStrategy.Builder recoveryStrategyBuilder;
 
   private volatile RecoveryStrategy recoveryStrat;
 
@@ -76,8 +77,15 @@ public final class DefaultSolrCoreState extends SolrCoreState implements Recover
   
   protected final ReentrantLock commitLock = new ReentrantLock();
 
+  @Deprecated
   public DefaultSolrCoreState(DirectoryFactory directoryFactory) {
+    this(directoryFactory, new RecoveryStrategy.Builder());
+  }
+
+  public DefaultSolrCoreState(DirectoryFactory directoryFactory,
+      RecoveryStrategy.Builder recoveryStrategyBuilder) {
     this.directoryFactory = directoryFactory;
+    this.recoveryStrategyBuilder = recoveryStrategyBuilder;
   }
   
   private void closeIndexWriter(IndexWriterCloser closer) {
@@ -263,6 +271,11 @@ public final class DefaultSolrCoreState extends SolrCoreState implements Recover
   }
 
   @Override
+  public RecoveryStrategy.Builder getRecoveryStrategyBuilder() {
+    return recoveryStrategyBuilder;
+  }
+
+  @Override
   public void doRecovery(CoreContainer cc, CoreDescriptor cd) {
     
     Thread thread = new Thread() {
@@ -310,7 +323,7 @@ public final class DefaultSolrCoreState extends SolrCoreState implements Recover
               recoveryThrottle.minimumWaitBetweenActions();
               recoveryThrottle.markAttemptingAction();
               
-              recoveryStrat = new RecoveryStrategy(cc, cd, DefaultSolrCoreState.this);
+              recoveryStrat = recoveryStrategyBuilder.create(cc, cd, DefaultSolrCoreState.this);
               recoveryStrat.setRecoveringAfterStartup(recoveringAfterStartup);
               Future<?> future = cc.getUpdateShardHandler().getRecoveryExecutor().submit(recoveryStrat);
               try {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c8bad8c1/solr/core/src/java/org/apache/solr/update/SolrCoreState.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/SolrCoreState.java b/solr/core/src/java/org/apache/solr/update/SolrCoreState.java
index f775b72..31dd66a 100644
--- a/solr/core/src/java/org/apache/solr/update/SolrCoreState.java
+++ b/solr/core/src/java/org/apache/solr/update/SolrCoreState.java
@@ -23,6 +23,7 @@ import java.util.concurrent.locks.Lock;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.search.Sort;
 import org.apache.solr.cloud.ActionThrottle;
+import org.apache.solr.cloud.RecoveryStrategy;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.core.DirectoryFactory;
@@ -144,6 +145,11 @@ public abstract class SolrCoreState {
    */
   public abstract DirectoryFactory getDirectoryFactory();
 
+  /**
+   * @return the {@link org.apache.solr.cloud.RecoveryStrategy.Builder} that should be used.
+   */
+  public abstract RecoveryStrategy.Builder getRecoveryStrategyBuilder();
+
 
   public interface IndexWriterCloser {
     void closeWriter(IndexWriter writer) throws IOException;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c8bad8c1/solr/core/src/test-files/solr/collection1/conf/solrconfig-configurerecoverystrategy.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-configurerecoverystrategy.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-configurerecoverystrategy.xml
new file mode 100644
index 0000000..62e671d
--- /dev/null
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-configurerecoverystrategy.xml
@@ -0,0 +1,28 @@
+<?xml version="1.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.
+-->
+
+<config>
+  <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <requestHandler name="standard" class="solr.StandardRequestHandler"></requestHandler>
+  <recoveryStrategy>
+    <int name="maxRetries">250</int>
+  </recoveryStrategy>
+  <schemaFactory class="ClassicIndexSchemaFactory"/>
+</config>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c8bad8c1/solr/core/src/test-files/solr/collection1/conf/solrconfig-customrecoverystrategy.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-customrecoverystrategy.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-customrecoverystrategy.xml
new file mode 100644
index 0000000..d43ed29
--- /dev/null
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-customrecoverystrategy.xml
@@ -0,0 +1,32 @@
+<?xml version="1.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.
+-->
+
+<config>
+  <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <requestHandler name="standard" class="solr.StandardRequestHandler"></requestHandler>
+  <!--
+    The RecoveryStrategy and RecoveryStrategy.Builder classes may change in future and customisations
+    are not supported between versions in terms of API or back compat behaviour.
+  -->
+  <recoveryStrategy class="org.apache.solr.core.ConfigureRecoveryStrategyTest$CustomRecoveryStrategyBuilder">
+    <str name="alternativeBaseUrlProp">recovery_base_url</str>
+  </recoveryStrategy>
+  <schemaFactory class="ClassicIndexSchemaFactory"/>
+</config>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c8bad8c1/solr/core/src/test/org/apache/solr/core/ConfigureRecoveryStrategyTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/ConfigureRecoveryStrategyTest.java b/solr/core/src/test/org/apache/solr/core/ConfigureRecoveryStrategyTest.java
new file mode 100644
index 0000000..80032af
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/core/ConfigureRecoveryStrategyTest.java
@@ -0,0 +1,111 @@
+/*
+ * 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.core;
+
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.cloud.RecoveryStrategy;
+import org.apache.solr.common.cloud.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.junit.BeforeClass;
+
+/**
+ * test that configs can override the RecoveryStrategy
+ */
+public class ConfigureRecoveryStrategyTest extends SolrTestCaseJ4 {
+
+  private static final String solrConfigFileNameConfigure = "solrconfig-configurerecoverystrategy.xml";
+  private static final String solrConfigFileNameCustom = "solrconfig-customrecoverystrategy.xml";
+
+  private static String solrConfigFileName;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    solrConfigFileName = (random().nextBoolean()
+        ? solrConfigFileNameConfigure : solrConfigFileNameCustom);
+    initCore(solrConfigFileName, "schema.xml");
+  }
+
+  public void testBuilder() throws Exception {
+    final RecoveryStrategy.Builder recoveryStrategyBuilder =
+        h.getCore().getSolrCoreState().getRecoveryStrategyBuilder();
+    assertNotNull("recoveryStrategyBuilder is null", recoveryStrategyBuilder);
+
+    final String expectedClassName;
+
+    if (solrConfigFileName.equals(solrConfigFileNameConfigure)) {
+      expectedClassName = RecoveryStrategy.Builder.class.getName();
+    } else if (solrConfigFileName.equals(solrConfigFileNameCustom)) {
+      assertTrue("recoveryStrategyBuilder is wrong class (instanceof)",
+          recoveryStrategyBuilder instanceof CustomRecoveryStrategyBuilder);
+      expectedClassName = ConfigureRecoveryStrategyTest.CustomRecoveryStrategyBuilder.class.getName();
+    } else {
+      expectedClassName = null;
+    }
+
+    assertEquals("recoveryStrategyBuilder is wrong class (name)",
+        expectedClassName, recoveryStrategyBuilder.getClass().getName());
+  }
+
+  public void testAlmostAllMethodsAreFinal() throws Exception {
+    for (Method m : RecoveryStrategy.class.getDeclaredMethods()) {
+      final String methodName = m.getName();
+      if ("getReplicateLeaderUrl".equals(methodName)) {
+        assertFalse(m.toString(), Modifier.isFinal(m.getModifiers()));
+      } else {
+        assertTrue(m.toString(), Modifier.isFinal(m.getModifiers()));
+      }
+    }
+  }
+
+  static public class CustomRecoveryStrategy extends RecoveryStrategy {
+
+    private String alternativeBaseUrlProp;
+
+    public String getAlternativeBaseUrlProp() {
+      return alternativeBaseUrlProp;
+    }
+
+    public void setAlternativeBaseUrlProp(String alternativeBaseUrlProp) {
+      this.alternativeBaseUrlProp = alternativeBaseUrlProp;
+    }
+
+    public CustomRecoveryStrategy(CoreContainer cc, CoreDescriptor cd,
+        RecoveryStrategy.RecoveryListener recoveryListener) {
+      super(cc, cd, recoveryListener);
+    }
+
+    @Override
+    protected String getReplicateLeaderUrl(ZkNodeProps leaderprops) {
+      return ZkCoreNodeProps.getCoreUrl(
+          leaderprops.getStr(alternativeBaseUrlProp),
+          leaderprops.getStr(ZkStateReader.CORE_NAME_PROP));
+    }
+  }
+
+  static public class CustomRecoveryStrategyBuilder extends RecoveryStrategy.Builder {
+    @Override
+    protected RecoveryStrategy newRecoveryStrategy(CoreContainer cc, CoreDescriptor cd,
+        RecoveryStrategy.RecoveryListener recoveryListener) {
+      return new CustomRecoveryStrategy(cc, cd, recoveryListener);
+    }
+  }
+
+}


[31/43] lucene-solr:feature/autoscaling: LUCENE-7740: Refactor Range Fields to remove Field suffix (e.g., DoubleRange), move InetAddressRange and InetAddressPoint from sandbox to misc module, and refactor all other range fields from sandbox to core.

Posted by sh...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/sandbox/src/test/org/apache/lucene/search/TestDoubleRangeFieldQueries.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/TestDoubleRangeFieldQueries.java b/lucene/sandbox/src/test/org/apache/lucene/search/TestDoubleRangeFieldQueries.java
deleted file mode 100644
index 43630e3..0000000
--- a/lucene/sandbox/src/test/org/apache/lucene/search/TestDoubleRangeFieldQueries.java
+++ /dev/null
@@ -1,251 +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.lucene.search;
-
-import java.util.Arrays;
-
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.DoubleRangeField;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.store.Directory;
-
-/**
- * Random testing for RangeFieldQueries.
- */
-public class TestDoubleRangeFieldQueries extends BaseRangeFieldQueryTestCase {
-  private static final String FIELD_NAME = "doubleRangeField";
-
-  private double nextDoubleInternal() {
-    if (rarely()) {
-      return random().nextBoolean() ? Double.POSITIVE_INFINITY : Double.NEGATIVE_INFINITY;
-    }
-    double max = Double.MAX_VALUE / 2;
-    return (max + max) * random().nextDouble() - max;
-  }
-
-  @Override
-  protected Range nextRange(int dimensions) {
-    double[] min = new double[dimensions];
-    double[] max = new double[dimensions];
-
-    double minV, maxV;
-    for (int d=0; d<dimensions; ++d) {
-      minV = nextDoubleInternal();
-      maxV = nextDoubleInternal();
-      min[d] = Math.min(minV, maxV);
-      max[d] = Math.max(minV, maxV);
-    }
-    return new DoubleRange(min, max);
-  }
-
-  @Override
-  protected DoubleRangeField newRangeField(Range r) {
-    return new DoubleRangeField(FIELD_NAME, ((DoubleRange)r).min, ((DoubleRange)r).max);
-  }
-
-  @Override
-  protected Query newIntersectsQuery(Range r) {
-    return DoubleRangeField.newIntersectsQuery(FIELD_NAME, ((DoubleRange)r).min, ((DoubleRange)r).max);
-  }
-
-  @Override
-  protected Query newContainsQuery(Range r) {
-    return DoubleRangeField.newContainsQuery(FIELD_NAME, ((DoubleRange)r).min, ((DoubleRange)r).max);
-  }
-
-  @Override
-  protected Query newWithinQuery(Range r) {
-    return DoubleRangeField.newWithinQuery(FIELD_NAME, ((DoubleRange)r).min, ((DoubleRange)r).max);
-  }
-
-  @Override
-  protected Query newCrossesQuery(Range r) {
-    return DoubleRangeField.newCrossesQuery(FIELD_NAME, ((DoubleRange)r).min, ((DoubleRange)r).max);
-  }
-
-  /** Basic test */
-  public void testBasics() throws Exception {
-    Directory dir = newDirectory();
-    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
-
-    // intersects (within)
-    Document document = new Document();
-    document.add(new DoubleRangeField(FIELD_NAME, new double[] {-10.0, -10.0}, new double[] {9.1, 10.1}));
-    writer.addDocument(document);
-
-    // intersects (crosses)
-    document = new Document();
-    document.add(new DoubleRangeField(FIELD_NAME, new double[] {10.0, -10.0}, new double[] {20.0, 10.0}));
-    writer.addDocument(document);
-
-    // intersects (contains, crosses)
-    document = new Document();
-    document.add(new DoubleRangeField(FIELD_NAME, new double[] {-20.0, -20.0}, new double[] {30.0, 30.1}));
-    writer.addDocument(document);
-
-    // intersects (crosses)
-    document = new Document();
-    document.add(new DoubleRangeField(FIELD_NAME, new double[] {-11.1, -11.2}, new double[] {1.23, 11.5}));
-    writer.addDocument(document);
-
-    // intersects (crosses)
-    document = new Document();
-    document.add(new DoubleRangeField(FIELD_NAME, new double[] {12.33, 1.2}, new double[] {15.1, 29.9}));
-    writer.addDocument(document);
-
-    // disjoint
-    document = new Document();
-    document.add(new DoubleRangeField(FIELD_NAME, new double[] {-122.33, 1.2}, new double[] {-115.1, 29.9}));
-    writer.addDocument(document);
-
-    // intersects (crosses)
-    document = new Document();
-    document.add(new DoubleRangeField(FIELD_NAME, new double[] {Double.NEGATIVE_INFINITY, 1.2}, new double[] {-11.0, 29.9}));
-    writer.addDocument(document);
-
-    // equal (within, contains, intersects)
-    document = new Document();
-    document.add(new DoubleRangeField(FIELD_NAME, new double[] {-11, -15}, new double[] {15, 20}));
-    writer.addDocument(document);
-
-    // search
-    IndexReader reader = writer.getReader();
-    IndexSearcher searcher = newSearcher(reader);
-    assertEquals(7, searcher.count(DoubleRangeField.newIntersectsQuery(FIELD_NAME,
-        new double[] {-11.0, -15.0}, new double[] {15.0, 20.0})));
-    assertEquals(2, searcher.count(DoubleRangeField.newWithinQuery(FIELD_NAME,
-        new double[] {-11.0, -15.0}, new double[] {15.0, 20.0})));
-    assertEquals(2, searcher.count(DoubleRangeField.newContainsQuery(FIELD_NAME,
-        new double[] {-11.0, -15.0}, new double[] {15.0, 20.0})));
-    assertEquals(5, searcher.count(DoubleRangeField.newCrossesQuery(FIELD_NAME,
-        new double[] {-11.0, -15.0}, new double[] {15.0, 20.0})));
-
-    reader.close();
-    writer.close();
-    dir.close();
-  }
-
-  /** DoubleRange test class implementation - use to validate DoubleRangeField */
-  private class DoubleRange extends Range {
-    double[] min;
-    double[] max;
-
-    DoubleRange(double[] min, double[] max) {
-      assert min != null && max != null && min.length > 0 && max.length > 0
-          : "test box: min/max cannot be null or empty";
-      assert min.length == max.length : "test box: min/max length do not agree";
-      this.min = min;
-      this.max = max;
-    }
-
-    @Override
-    protected int numDimensions() {
-      return min.length;
-    }
-
-    @Override
-    protected Double getMin(int dim) {
-      return min[dim];
-    }
-
-    @Override
-    protected void setMin(int dim, Object val) {
-      double v = (Double)val;
-      if (min[dim] < v) {
-        max[dim] = v;
-      } else {
-        min[dim] = v;
-      }
-    }
-
-    @Override
-    protected Double getMax(int dim) {
-      return max[dim];
-    }
-
-    @Override
-    protected void setMax(int dim, Object val) {
-      double v = (Double)val;
-      if (max[dim] > v) {
-        min[dim] = v;
-      } else {
-        max[dim] = v;
-      }
-    }
-
-    @Override
-    protected boolean isEqual(Range other) {
-      DoubleRange o = (DoubleRange)other;
-      return Arrays.equals(min, o.min) && Arrays.equals(max, o.max);
-    }
-
-    @Override
-    protected boolean isDisjoint(Range o) {
-      DoubleRange other = (DoubleRange)o;
-      for (int d=0; d<this.min.length; ++d) {
-        if (this.min[d] > other.max[d] || this.max[d] < other.min[d]) {
-          // disjoint:
-          return true;
-        }
-      }
-      return false;
-    }
-
-    @Override
-    protected boolean isWithin(Range o) {
-      DoubleRange other = (DoubleRange)o;
-      for (int d=0; d<this.min.length; ++d) {
-        if ((this.min[d] >= other.min[d] && this.max[d] <= other.max[d]) == false) {
-          // not within:
-          return false;
-        }
-      }
-      return true;
-    }
-
-    @Override
-    protected boolean contains(Range o) {
-      DoubleRange other = (DoubleRange) o;
-      for (int d=0; d<this.min.length; ++d) {
-        if ((this.min[d] <= other.min[d] && this.max[d] >= other.max[d]) == false) {
-          // not contains:
-          return false;
-        }
-      }
-      return true;
-    }
-
-    @Override
-    public String toString() {
-      StringBuilder b = new StringBuilder();
-      b.append("Box(");
-      b.append(min[0]);
-      b.append(" TO ");
-      b.append(max[0]);
-      for (int d=1; d<min.length; ++d) {
-        b.append(", ");
-        b.append(min[d]);
-        b.append(" TO ");
-        b.append(max[d]);
-      }
-      b.append(")");
-
-      return b.toString();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/sandbox/src/test/org/apache/lucene/search/TestFloatRangeFieldQueries.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/TestFloatRangeFieldQueries.java b/lucene/sandbox/src/test/org/apache/lucene/search/TestFloatRangeFieldQueries.java
deleted file mode 100644
index 3509e35..0000000
--- a/lucene/sandbox/src/test/org/apache/lucene/search/TestFloatRangeFieldQueries.java
+++ /dev/null
@@ -1,251 +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.lucene.search;
-
-import java.util.Arrays;
-
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.FloatRangeField;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.store.Directory;
-
-/**
- * Random testing for FloatRangeField Queries.
- */
-public class TestFloatRangeFieldQueries extends BaseRangeFieldQueryTestCase {
-  private static final String FIELD_NAME = "floatRangeField";
-
-  private float nextFloatInternal() {
-    if (rarely()) {
-      return random().nextBoolean() ? Float.NEGATIVE_INFINITY : Float.POSITIVE_INFINITY;
-    }
-    float max = Float.MAX_VALUE / 2;
-    return (max + max) * random().nextFloat() - max;
-  }
-
-  @Override
-  protected Range nextRange(int dimensions) {
-    float[] min = new float[dimensions];
-    float[] max = new float[dimensions];
-
-    float minV, maxV;
-    for (int d=0; d<dimensions; ++d) {
-      minV = nextFloatInternal();
-      maxV = nextFloatInternal();
-      min[d] = Math.min(minV, maxV);
-      max[d] = Math.max(minV, maxV);
-    }
-    return new FloatRange(min, max);
-  }
-
-  @Override
-  protected FloatRangeField newRangeField(Range r) {
-    return new FloatRangeField(FIELD_NAME, ((FloatRange)r).min, ((FloatRange)r).max);
-  }
-
-  @Override
-  protected Query newIntersectsQuery(Range r) {
-    return FloatRangeField.newIntersectsQuery(FIELD_NAME, ((FloatRange)r).min, ((FloatRange)r).max);
-  }
-
-  @Override
-  protected Query newContainsQuery(Range r) {
-    return FloatRangeField.newContainsQuery(FIELD_NAME, ((FloatRange)r).min, ((FloatRange)r).max);
-  }
-
-  @Override
-  protected Query newWithinQuery(Range r) {
-    return FloatRangeField.newWithinQuery(FIELD_NAME, ((FloatRange)r).min, ((FloatRange)r).max);
-  }
-
-  @Override
-  protected Query newCrossesQuery(Range r) {
-    return FloatRangeField.newCrossesQuery(FIELD_NAME, ((FloatRange)r).min, ((FloatRange)r).max);
-  }
-
-  /** Basic test */
-  public void testBasics() throws Exception {
-    Directory dir = newDirectory();
-    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
-
-    // intersects (within)
-    Document document = new Document();
-    document.add(new FloatRangeField(FIELD_NAME, new float[] {-10.0f, -10.0f}, new float[] {9.1f, 10.1f}));
-    writer.addDocument(document);
-
-    // intersects (crosses)
-    document = new Document();
-    document.add(new FloatRangeField(FIELD_NAME, new float[] {10.0f, -10.0f}, new float[] {20.0f, 10.0f}));
-    writer.addDocument(document);
-
-    // intersects (contains, crosses)
-    document = new Document();
-    document.add(new FloatRangeField(FIELD_NAME, new float[] {-20.0f, -20.0f}, new float[] {30.0f, 30.1f}));
-    writer.addDocument(document);
-
-    // intersects (crosses)
-    document = new Document();
-    document.add(new FloatRangeField(FIELD_NAME, new float[] {-11.1f, -11.2f}, new float[] {1.23f, 11.5f}));
-    writer.addDocument(document);
-
-    // intersects (crosses)
-    document = new Document();
-    document.add(new FloatRangeField(FIELD_NAME, new float[] {12.33f, 1.2f}, new float[] {15.1f, 29.9f}));
-    writer.addDocument(document);
-
-    // disjoint
-    document = new Document();
-    document.add(new FloatRangeField(FIELD_NAME, new float[] {-122.33f, 1.2f}, new float[] {-115.1f, 29.9f}));
-    writer.addDocument(document);
-
-    // intersects (crosses)
-    document = new Document();
-    document.add(new FloatRangeField(FIELD_NAME, new float[] {Float.NEGATIVE_INFINITY, 1.2f}, new float[] {-11.0f, 29.9f}));
-    writer.addDocument(document);
-
-    // equal (within, contains, intersects)
-    document = new Document();
-    document.add(new FloatRangeField(FIELD_NAME, new float[] {-11f, -15f}, new float[] {15f, 20f}));
-    writer.addDocument(document);
-
-    // search
-    IndexReader reader = writer.getReader();
-    IndexSearcher searcher = newSearcher(reader);
-    assertEquals(7, searcher.count(FloatRangeField.newIntersectsQuery(FIELD_NAME,
-        new float[] {-11.0f, -15.0f}, new float[] {15.0f, 20.0f})));
-    assertEquals(2, searcher.count(FloatRangeField.newWithinQuery(FIELD_NAME,
-        new float[] {-11.0f, -15.0f}, new float[] {15.0f, 20.0f})));
-    assertEquals(2, searcher.count(FloatRangeField.newContainsQuery(FIELD_NAME,
-        new float[] {-11.0f, -15.0f}, new float[] {15.0f, 20.0f})));
-    assertEquals(5, searcher.count(FloatRangeField.newCrossesQuery(FIELD_NAME,
-        new float[] {-11.0f, -15.0f}, new float[] {15.0f, 20.0f})));
-
-    reader.close();
-    writer.close();
-    dir.close();
-  }
-
-  /** FloatRange test class implementation - use to validate FloatRangeField */
-  private class FloatRange extends Range {
-    float[] min;
-    float[] max;
-
-    FloatRange(float[] min, float[] max) {
-      assert min != null && max != null && min.length > 0 && max.length > 0
-          : "test box: min/max cannot be null or empty";
-      assert min.length == max.length : "test box: min/max length do not agree";
-      this.min = min;
-      this.max = max;
-    }
-
-    @Override
-    protected int numDimensions() {
-      return min.length;
-    }
-
-    @Override
-    protected Float getMin(int dim) {
-      return min[dim];
-    }
-
-    @Override
-    protected void setMin(int dim, Object val) {
-      float v = (Float)val;
-      if (min[dim] < v) {
-        max[dim] = v;
-      } else {
-        min[dim] = v;
-      }
-    }
-
-    @Override
-    protected Float getMax(int dim) {
-      return max[dim];
-    }
-
-    @Override
-    protected void setMax(int dim, Object val) {
-      float v = (Float)val;
-      if (max[dim] > v) {
-        min[dim] = v;
-      } else {
-        max[dim] = v;
-      }
-    }
-
-    @Override
-    protected boolean isEqual(Range other) {
-      FloatRange o = (FloatRange)other;
-      return Arrays.equals(min, o.min) && Arrays.equals(max, o.max);
-    }
-
-    @Override
-    protected boolean isDisjoint(Range o) {
-      FloatRange other = (FloatRange)o;
-      for (int d=0; d<this.min.length; ++d) {
-        if (this.min[d] > other.max[d] || this.max[d] < other.min[d]) {
-          // disjoint:
-          return true;
-        }
-      }
-      return false;
-    }
-
-    @Override
-    protected boolean isWithin(Range o) {
-      FloatRange other = (FloatRange)o;
-      for (int d=0; d<this.min.length; ++d) {
-        if ((this.min[d] >= other.min[d] && this.max[d] <= other.max[d]) == false) {
-          // not within:
-          return false;
-        }
-      }
-      return true;
-    }
-
-    @Override
-    protected boolean contains(Range o) {
-      FloatRange other = (FloatRange) o;
-      for (int d=0; d<this.min.length; ++d) {
-        if ((this.min[d] <= other.min[d] && this.max[d] >= other.max[d]) == false) {
-          // not contains:
-          return false;
-        }
-      }
-      return true;
-    }
-
-    @Override
-    public String toString() {
-      StringBuilder b = new StringBuilder();
-      b.append("Box(");
-      b.append(min[0]);
-      b.append(" TO ");
-      b.append(max[0]);
-      for (int d=1; d<min.length; ++d) {
-        b.append(", ");
-        b.append(min[d]);
-        b.append(" TO ");
-        b.append(max[d]);
-      }
-      b.append(")");
-
-      return b.toString();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/sandbox/src/test/org/apache/lucene/search/TestIntRangeFieldQueries.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/TestIntRangeFieldQueries.java b/lucene/sandbox/src/test/org/apache/lucene/search/TestIntRangeFieldQueries.java
deleted file mode 100644
index 0bb782e..0000000
--- a/lucene/sandbox/src/test/org/apache/lucene/search/TestIntRangeFieldQueries.java
+++ /dev/null
@@ -1,251 +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.lucene.search;
-
-import java.util.Arrays;
-
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.IntRangeField;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.store.Directory;
-
-/**
- * Random testing for IntRangeField Queries.
- */
-public class TestIntRangeFieldQueries extends BaseRangeFieldQueryTestCase {
-  private static final String FIELD_NAME = "intRangeField";
-
-  private int nextIntInternal() {
-    if (rarely()) {
-      return random().nextBoolean() ? Integer.MAX_VALUE : Integer.MIN_VALUE;
-    }
-    int max = Integer.MAX_VALUE / 2;
-    return (max + max) * random().nextInt() - max;
-  }
-
-  @Override
-  protected Range nextRange(int dimensions) {
-    int[] min = new int[dimensions];
-    int[] max = new int[dimensions];
-
-    int minV, maxV;
-    for (int d=0; d<dimensions; ++d) {
-      minV = nextIntInternal();
-      maxV = nextIntInternal();
-      min[d] = Math.min(minV, maxV);
-      max[d] = Math.max(minV, maxV);
-    }
-    return new IntRange(min, max);
-  }
-
-  @Override
-  protected IntRangeField newRangeField(Range r) {
-    return new IntRangeField(FIELD_NAME, ((IntRange)r).min, ((IntRange)r).max);
-  }
-
-  @Override
-  protected Query newIntersectsQuery(Range r) {
-    return IntRangeField.newIntersectsQuery(FIELD_NAME, ((IntRange)r).min, ((IntRange)r).max);
-  }
-
-  @Override
-  protected Query newContainsQuery(Range r) {
-    return IntRangeField.newContainsQuery(FIELD_NAME, ((IntRange)r).min, ((IntRange)r).max);
-  }
-
-  @Override
-  protected Query newWithinQuery(Range r) {
-    return IntRangeField.newWithinQuery(FIELD_NAME, ((IntRange)r).min, ((IntRange)r).max);
-  }
-
-  @Override
-  protected Query newCrossesQuery(Range r) {
-    return IntRangeField.newCrossesQuery(FIELD_NAME, ((IntRange)r).min, ((IntRange)r).max);
-  }
-
-  /** Basic test */
-  public void testBasics() throws Exception {
-    Directory dir = newDirectory();
-    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
-
-    // intersects (within)
-    Document document = new Document();
-    document.add(new IntRangeField(FIELD_NAME, new int[] {-10, -10}, new int[] {9, 10}));
-    writer.addDocument(document);
-
-    // intersects (crosses)
-    document = new Document();
-    document.add(new IntRangeField(FIELD_NAME, new int[] {10, -10}, new int[] {20, 10}));
-    writer.addDocument(document);
-
-    // intersects (contains / crosses)
-    document = new Document();
-    document.add(new IntRangeField(FIELD_NAME, new int[] {-20, -20}, new int[] {30, 30}));
-    writer.addDocument(document);
-
-    // intersects (within)
-    document = new Document();
-    document.add(new IntRangeField(FIELD_NAME, new int[] {-11, -11}, new int[] {1, 11}));
-    writer.addDocument(document);
-
-    // intersects (crosses)
-    document = new Document();
-    document.add(new IntRangeField(FIELD_NAME, new int[] {12, 1}, new int[] {15, 29}));
-    writer.addDocument(document);
-
-    // disjoint
-    document = new Document();
-    document.add(new IntRangeField(FIELD_NAME, new int[] {-122, 1}, new int[] {-115, 29}));
-    writer.addDocument(document);
-
-    // intersects (crosses)
-    document = new Document();
-    document.add(new IntRangeField(FIELD_NAME, new int[] {Integer.MIN_VALUE, 1}, new int[] {-11, 29}));
-    writer.addDocument(document);
-
-    // equal (within, contains, intersects)
-    document = new Document();
-    document.add(new IntRangeField(FIELD_NAME, new int[] {-11, -15}, new int[] {15, 20}));
-    writer.addDocument(document);
-
-    // search
-    IndexReader reader = writer.getReader();
-    IndexSearcher searcher = newSearcher(reader);
-    assertEquals(7, searcher.count(IntRangeField.newIntersectsQuery(FIELD_NAME,
-        new int[] {-11, -15}, new int[] {15, 20})));
-    assertEquals(3, searcher.count(IntRangeField.newWithinQuery(FIELD_NAME,
-        new int[] {-11, -15}, new int[] {15, 20})));
-    assertEquals(2, searcher.count(IntRangeField.newContainsQuery(FIELD_NAME,
-        new int[] {-11, -15}, new int[] {15, 20})));
-    assertEquals(4, searcher.count(IntRangeField.newCrossesQuery(FIELD_NAME,
-        new int[] {-11, -15}, new int[] {15, 20})));
-
-    reader.close();
-    writer.close();
-    dir.close();
-  }
-
-  /** IntRange test class implementation - use to validate IntRangeField */
-  private class IntRange extends Range {
-    int[] min;
-    int[] max;
-
-    IntRange(int[] min, int[] max) {
-      assert min != null && max != null && min.length > 0 && max.length > 0
-          : "test box: min/max cannot be null or empty";
-      assert min.length == max.length : "test box: min/max length do not agree";
-      this.min = min;
-      this.max = max;
-    }
-
-    @Override
-    protected int numDimensions() {
-      return min.length;
-    }
-
-    @Override
-    protected Integer getMin(int dim) {
-      return min[dim];
-    }
-
-    @Override
-    protected void setMin(int dim, Object val) {
-      int v = (Integer)val;
-      if (min[dim] < v) {
-        max[dim] = v;
-      } else {
-        min[dim] = v;
-      }
-    }
-
-    @Override
-    protected Integer getMax(int dim) {
-      return max[dim];
-    }
-
-    @Override
-    protected void setMax(int dim, Object val) {
-      int v = (Integer)val;
-      if (max[dim] > v) {
-        min[dim] = v;
-      } else {
-        max[dim] = v;
-      }
-    }
-
-    @Override
-    protected boolean isEqual(Range other) {
-      IntRange o = (IntRange)other;
-      return Arrays.equals(min, o.min) && Arrays.equals(max, o.max);
-    }
-
-    @Override
-    protected boolean isDisjoint(Range o) {
-      IntRange other = (IntRange)o;
-      for (int d=0; d<this.min.length; ++d) {
-        if (this.min[d] > other.max[d] || this.max[d] < other.min[d]) {
-          // disjoint:
-          return true;
-        }
-      }
-      return false;
-    }
-
-    @Override
-    protected boolean isWithin(Range o) {
-      IntRange other = (IntRange)o;
-      for (int d=0; d<this.min.length; ++d) {
-        if ((this.min[d] >= other.min[d] && this.max[d] <= other.max[d]) == false) {
-          // not within:
-          return false;
-        }
-      }
-      return true;
-    }
-
-    @Override
-    protected boolean contains(Range o) {
-      IntRange other = (IntRange) o;
-      for (int d=0; d<this.min.length; ++d) {
-        if ((this.min[d] <= other.min[d] && this.max[d] >= other.max[d]) == false) {
-          // not contains:
-          return false;
-        }
-      }
-      return true;
-    }
-
-    @Override
-    public String toString() {
-      StringBuilder b = new StringBuilder();
-      b.append("Box(");
-      b.append(min[0]);
-      b.append(" TO ");
-      b.append(max[0]);
-      for (int d=1; d<min.length; ++d) {
-        b.append(", ");
-        b.append(min[d]);
-        b.append(" TO ");
-        b.append(max[d]);
-      }
-      b.append(")");
-
-      return b.toString();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/sandbox/src/test/org/apache/lucene/search/TestIpRangeFieldQueries.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/TestIpRangeFieldQueries.java b/lucene/sandbox/src/test/org/apache/lucene/search/TestIpRangeFieldQueries.java
deleted file mode 100644
index 1563584..0000000
--- a/lucene/sandbox/src/test/org/apache/lucene/search/TestIpRangeFieldQueries.java
+++ /dev/null
@@ -1,220 +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.lucene.search;
-
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-
-import org.apache.lucene.document.InetAddressRangeField;
-import org.apache.lucene.util.StringHelper;
-
-/**
- * Random testing for {@link InetAddressRangeField}
- */
-public class TestIpRangeFieldQueries extends BaseRangeFieldQueryTestCase {
-  private static final String FIELD_NAME = "ipRangeField";
-
-  private IPVersion ipVersion;
-
-  private enum IPVersion {IPv4, IPv6}
-
-  @Override
-  protected Range nextRange(int dimensions) {
-    try {
-      InetAddress min = nextInetaddress();
-      byte[] bMin = min.getAddress();
-      InetAddress max = nextInetaddress();
-      byte[] bMax = max.getAddress();
-      if (StringHelper.compare(bMin.length, bMin, 0, bMax, 0) > 0) {
-        return new IpRange(max, min);
-      }
-      return new IpRange(min, max);
-    } catch (UnknownHostException e) {
-      e.printStackTrace();
-    }
-    return null;
-  }
-
-  /** return random IPv4 or IPv6 address */
-  private InetAddress nextInetaddress() throws UnknownHostException {
-    byte[] b;
-    switch (ipVersion) {
-      case IPv4:
-        b = new byte[4];
-        break;
-      case IPv6:
-        b = new byte[16];
-        break;
-      default:
-        throw new IllegalArgumentException("incorrect IP version: " + ipVersion);
-    }
-    random().nextBytes(b);
-    return InetAddress.getByAddress(b);
-  }
-
-  /** randomly select version across tests */
-  private IPVersion ipVersion() {
-    return random().nextBoolean() ? IPVersion.IPv4 : IPVersion.IPv6;
-  }
-
-  @Override
-  public void testRandomTiny() throws Exception {
-    ipVersion = ipVersion();
-    super.testRandomTiny();
-  }
-
-  @Override
-  public void testMultiValued() throws Exception {
-    ipVersion = ipVersion();
-    super.testRandomMedium();
-  }
-
-  @Override
-  public void testRandomMedium() throws Exception {
-    ipVersion = ipVersion();
-    super.testMultiValued();
-  }
-
-  @Nightly
-  @Override
-  public void testRandomBig() throws Exception {
-    ipVersion = ipVersion();
-    super.testRandomBig();
-  }
-
-  /** return random range */
-  @Override
-  protected InetAddressRangeField newRangeField(Range r) {
-    return new InetAddressRangeField(FIELD_NAME, ((IpRange)r).min, ((IpRange)r).max);
-  }
-
-  /** return random intersects query */
-  @Override
-  protected Query newIntersectsQuery(Range r) {
-    return InetAddressRangeField.newIntersectsQuery(FIELD_NAME, ((IpRange)r).min, ((IpRange)r).max);
-  }
-
-  /** return random contains query */
-  @Override
-  protected Query newContainsQuery(Range r) {
-    return InetAddressRangeField.newContainsQuery(FIELD_NAME, ((IpRange)r).min, ((IpRange)r).max);
-  }
-
-  /** return random within query */
-  @Override
-  protected Query newWithinQuery(Range r) {
-    return InetAddressRangeField.newWithinQuery(FIELD_NAME, ((IpRange)r).min, ((IpRange)r).max);
-  }
-
-  /** return random crosses query */
-  @Override
-  protected Query newCrossesQuery(Range r) {
-    return InetAddressRangeField.newCrossesQuery(FIELD_NAME, ((IpRange)r).min, ((IpRange)r).max);
-  }
-
-  /** encapsulated IpRange for test validation */
-  private class IpRange extends Range {
-    InetAddress min;
-    InetAddress max;
-
-    IpRange(InetAddress min, InetAddress max) {
-      this.min = min;
-      this.max = max;
-    }
-
-    @Override
-    protected int numDimensions() {
-      return 1;
-    }
-
-    @Override
-    protected InetAddress getMin(int dim) {
-      return min;
-    }
-
-    @Override
-    protected void setMin(int dim, Object val) {
-      byte[] v = ((InetAddress)val).getAddress();
-
-      if (StringHelper.compare(v.length, min.getAddress(), 0, v, 0) < 0) {
-        max = (InetAddress)val;
-      } else {
-        min = (InetAddress) val;
-      }
-    }
-
-    @Override
-    protected InetAddress getMax(int dim) {
-      return max;
-    }
-
-    @Override
-    protected void setMax(int dim, Object val) {
-      byte[] v = ((InetAddress)val).getAddress();
-
-      if (StringHelper.compare(v.length, max.getAddress(), 0, v, 0) > 0) {
-        min = (InetAddress)val;
-      } else {
-        max = (InetAddress) val;
-      }
-    }
-
-    @Override
-    protected boolean isEqual(Range o) {
-      IpRange other = (IpRange)o;
-      return this.min.equals(other.min) && this.max.equals(other.max);
-    }
-
-    @Override
-    protected boolean isDisjoint(Range o) {
-      IpRange other = (IpRange)o;
-      byte[] bMin = min.getAddress();
-      byte[] bMax = max.getAddress();
-      return StringHelper.compare(bMin.length, bMin, 0, other.max.getAddress(), 0) > 0 ||
-          StringHelper.compare(bMax.length, bMax, 0, other.min.getAddress(), 0) < 0;
-    }
-
-    @Override
-    protected boolean isWithin(Range o) {
-      IpRange other = (IpRange)o;
-      byte[] bMin = min.getAddress();
-      byte[] bMax = max.getAddress();
-      return StringHelper.compare(bMin.length, bMin, 0, other.min.getAddress(), 0) >= 0 &&
-          StringHelper.compare(bMax.length, bMax, 0, other.max.getAddress(), 0) <= 0;
-    }
-
-    @Override
-    protected boolean contains(Range o) {
-      IpRange other = (IpRange)o;
-      byte[] bMin = min.getAddress();
-      byte[] bMax = max.getAddress();
-      return StringHelper.compare(bMin.length, bMin, 0, other.min.getAddress(), 0) <= 0 &&
-          StringHelper.compare(bMax.length, bMax, 0, other.max.getAddress(), 0) >= 0;
-    }
-
-    @Override
-    public String toString() {
-      StringBuilder b = new StringBuilder();
-      b.append("Box(");
-      b.append(min.getHostAddress());
-      b.append(" TO ");
-      b.append(max.getHostAddress());
-      b.append(")");
-      return b.toString();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/sandbox/src/test/org/apache/lucene/search/TestLongRangeFieldQueries.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/TestLongRangeFieldQueries.java b/lucene/sandbox/src/test/org/apache/lucene/search/TestLongRangeFieldQueries.java
deleted file mode 100644
index fc21a64..0000000
--- a/lucene/sandbox/src/test/org/apache/lucene/search/TestLongRangeFieldQueries.java
+++ /dev/null
@@ -1,251 +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.lucene.search;
-
-import java.util.Arrays;
-
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.LongRangeField;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.store.Directory;
-
-/**
- * Random testing for LongRangeField Queries.
- */
-public class TestLongRangeFieldQueries extends BaseRangeFieldQueryTestCase {
-  private static final String FIELD_NAME = "longRangeField";
-
-  private long nextLongInternal() {
-    if (rarely()) {
-      return random().nextBoolean() ? Long.MAX_VALUE : Long.MIN_VALUE;
-    }
-    long max = Long.MAX_VALUE / 2;
-    return (max + max) * random().nextLong() - max;
-  }
-
-  @Override
-  protected Range nextRange(int dimensions) {
-    long[] min = new long[dimensions];
-    long[] max = new long[dimensions];
-
-    long minV, maxV;
-    for (int d=0; d<dimensions; ++d) {
-      minV = nextLongInternal();
-      maxV = nextLongInternal();
-      min[d] = Math.min(minV, maxV);
-      max[d] = Math.max(minV, maxV);
-    }
-    return new LongRange(min, max);
-  }
-
-  @Override
-  protected LongRangeField newRangeField(Range r) {
-    return new LongRangeField(FIELD_NAME, ((LongRange)r).min, ((LongRange)r).max);
-  }
-
-  @Override
-  protected Query newIntersectsQuery(Range r) {
-    return LongRangeField.newIntersectsQuery(FIELD_NAME, ((LongRange)r).min, ((LongRange)r).max);
-  }
-
-  @Override
-  protected Query newContainsQuery(Range r) {
-    return LongRangeField.newContainsQuery(FIELD_NAME, ((LongRange)r).min, ((LongRange)r).max);
-  }
-
-  @Override
-  protected Query newWithinQuery(Range r) {
-    return LongRangeField.newWithinQuery(FIELD_NAME, ((LongRange)r).min, ((LongRange)r).max);
-  }
-
-  @Override
-  protected Query newCrossesQuery(Range r) {
-    return LongRangeField.newCrossesQuery(FIELD_NAME, ((LongRange)r).min, ((LongRange)r).max);
-  }
-
-  /** Basic test */
-  public void testBasics() throws Exception {
-    Directory dir = newDirectory();
-    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
-
-    // intersects (within)
-    Document document = new Document();
-    document.add(new LongRangeField(FIELD_NAME, new long[] {-10, -10}, new long[] {9, 10}));
-    writer.addDocument(document);
-
-    // intersects (crosses)
-    document = new Document();
-    document.add(new LongRangeField(FIELD_NAME, new long[] {10, -10}, new long[] {20, 10}));
-    writer.addDocument(document);
-
-    // intersects (contains, crosses)
-    document = new Document();
-    document.add(new LongRangeField(FIELD_NAME, new long[] {-20, -20}, new long[] {30, 30}));
-    writer.addDocument(document);
-
-    // intersects (within)
-    document = new Document();
-    document.add(new LongRangeField(FIELD_NAME, new long[] {-11, -11}, new long[] {1, 11}));
-    writer.addDocument(document);
-
-    // intersects (crosses)
-    document = new Document();
-    document.add(new LongRangeField(FIELD_NAME, new long[] {12, 1}, new long[] {15, 29}));
-    writer.addDocument(document);
-
-    // disjoint
-    document = new Document();
-    document.add(new LongRangeField(FIELD_NAME, new long[] {-122, 1}, new long[] {-115, 29}));
-    writer.addDocument(document);
-
-    // intersects (crosses)
-    document = new Document();
-    document.add(new LongRangeField(FIELD_NAME, new long[] {Long.MIN_VALUE, 1}, new long[] {-11, 29}));
-    writer.addDocument(document);
-
-    // equal (within, contains, intersects)
-    document = new Document();
-    document.add(new LongRangeField(FIELD_NAME, new long[] {-11, -15}, new long[] {15, 20}));
-    writer.addDocument(document);
-
-    // search
-    IndexReader reader = writer.getReader();
-    IndexSearcher searcher = newSearcher(reader);
-    assertEquals(7, searcher.count(LongRangeField.newIntersectsQuery(FIELD_NAME,
-        new long[] {-11, -15}, new long[] {15, 20})));
-    assertEquals(3, searcher.count(LongRangeField.newWithinQuery(FIELD_NAME,
-        new long[] {-11, -15}, new long[] {15, 20})));
-    assertEquals(2, searcher.count(LongRangeField.newContainsQuery(FIELD_NAME,
-        new long[] {-11, -15}, new long[] {15, 20})));
-    assertEquals(4, searcher.count(LongRangeField.newCrossesQuery(FIELD_NAME,
-        new long[] {-11, -15}, new long[] {15, 20})));
-
-    reader.close();
-    writer.close();
-    dir.close();
-  }
-
-  /** LongRange test class implementation - use to validate LongRangeField */
-  private class LongRange extends Range {
-    long[] min;
-    long[] max;
-
-    LongRange(long[] min, long[] max) {
-      assert min != null && max != null && min.length > 0 && max.length > 0
-          : "test box: min/max cannot be null or empty";
-      assert min.length == max.length : "test box: min/max length do not agree";
-      this.min = min;
-      this.max = max;
-    }
-
-    @Override
-    protected int numDimensions() {
-      return min.length;
-    }
-
-    @Override
-    protected Long getMin(int dim) {
-      return min[dim];
-    }
-
-    @Override
-    protected void setMin(int dim, Object val) {
-      long v = (Long)val;
-      if (min[dim] < v) {
-        max[dim] = v;
-      } else {
-        min[dim] = v;
-      }
-    }
-
-    @Override
-    protected Long getMax(int dim) {
-      return max[dim];
-    }
-
-    @Override
-    protected void setMax(int dim, Object val) {
-      long v = (Long)val;
-      if (max[dim] > v) {
-        min[dim] = v;
-      } else {
-        max[dim] = v;
-      }
-    }
-
-    @Override
-    protected boolean isEqual(Range other) {
-      LongRange o = (LongRange)other;
-      return Arrays.equals(min, o.min) && Arrays.equals(max, o.max);
-    }
-
-    @Override
-    protected boolean isDisjoint(Range o) {
-      LongRange other = (LongRange)o;
-      for (int d=0; d<this.min.length; ++d) {
-        if (this.min[d] > other.max[d] || this.max[d] < other.min[d]) {
-          // disjoint:
-          return true;
-        }
-      }
-      return false;
-    }
-
-    @Override
-    protected boolean isWithin(Range o) {
-      LongRange other = (LongRange)o;
-      for (int d=0; d<this.min.length; ++d) {
-        if ((this.min[d] >= other.min[d] && this.max[d] <= other.max[d]) == false) {
-          // not within:
-          return false;
-        }
-      }
-      return true;
-    }
-
-    @Override
-    protected boolean contains(Range o) {
-      LongRange other = (LongRange) o;
-      for (int d=0; d<this.min.length; ++d) {
-        if ((this.min[d] <= other.min[d] && this.max[d] >= other.max[d]) == false) {
-          // not contains:
-          return false;
-        }
-      }
-      return true;
-    }
-
-    @Override
-    public String toString() {
-      StringBuilder b = new StringBuilder();
-      b.append("Box(");
-      b.append(min[0]);
-      b.append(" TO ");
-      b.append(max[0]);
-      for (int d=1; d<min.length; ++d) {
-        b.append(", ");
-        b.append(min[d]);
-        b.append(" TO ");
-        b.append(max[d]);
-      }
-      b.append(")");
-
-      return b.toString();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/test-framework/src/java/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java
new file mode 100644
index 0000000..76de732
--- /dev/null
+++ b/lucene/test-framework/src/java/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java
@@ -0,0 +1,346 @@
+/*
+ * 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.HashSet;
+import java.util.Set;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.MultiDocValues;
+import org.apache.lucene.index.MultiFields;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.SerialMergeScheduler;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LuceneTestCase;
+
+/**
+ * Abstract class to do basic tests for a RangeField query. Testing rigor inspired by {@code BaseGeoPointTestCase}
+ */
+public abstract class BaseRangeFieldQueryTestCase extends LuceneTestCase {
+  protected abstract Field newRangeField(Range box);
+
+  protected abstract Query newIntersectsQuery(Range box);
+
+  protected abstract Query newContainsQuery(Range box);
+
+  protected abstract Query newWithinQuery(Range box);
+
+  protected abstract Query newCrossesQuery(Range box);
+
+  protected abstract Range nextRange(int dimensions) throws Exception;
+
+  protected int dimension() {
+    return random().nextInt(4) + 1;
+  }
+
+  public void testRandomTiny() throws Exception {
+    // Make sure single-leaf-node case is OK:
+    doTestRandom(10, false);
+  }
+
+  public void testRandomMedium() throws Exception {
+    doTestRandom(10000, false);
+  }
+
+  @Nightly
+  public void testRandomBig() throws Exception {
+    doTestRandom(200000, false);
+  }
+
+  public void testMultiValued() throws Exception {
+    doTestRandom(10000, true);
+  }
+
+  private void doTestRandom(int count, boolean multiValued) throws Exception {
+    int numDocs = atLeast(count);
+    int dimensions = dimension();
+
+    if (VERBOSE) {
+      System.out.println("TEST: numDocs=" + numDocs);
+    }
+
+    Range[][] ranges = new Range[numDocs][];
+
+    boolean haveRealDoc = true;
+
+    nextdoc: for (int id=0; id<numDocs; ++id) {
+      int x = random().nextInt(20);
+      if (ranges[id] == null) {
+        ranges[id] = new Range[] {nextRange(dimensions)};
+      }
+      if (x == 17) {
+        // some docs don't have a box:
+        ranges[id][0].isMissing = true;
+        if (VERBOSE) {
+          System.out.println("  id=" + id + " is missing");
+        }
+        continue;
+      }
+
+      if (multiValued == true && random().nextBoolean()) {
+        // randomly add multi valued documents (up to 2 fields)
+        int n = random().nextInt(2) + 1;
+        ranges[id] = new Range[n];
+        for (int i=0; i<n; ++i) {
+          ranges[id][i] = nextRange(dimensions);
+        }
+      }
+
+      if (id > 0 && x < 9 && haveRealDoc) {
+        int oldID;
+        int i=0;
+        // don't step on missing ranges:
+        while (true) {
+          oldID = random().nextInt(id);
+          if (ranges[oldID][0].isMissing == false) {
+            break;
+          } else if (++i > id) {
+            continue nextdoc;
+          }
+        }
+
+        if (x == dimensions*2) {
+          // Fully identical box (use first box in case current is multivalued but old is not)
+          for (int d=0; d<dimensions; ++d) {
+            ranges[id][0].setMin(d, ranges[oldID][0].getMin(d));
+            ranges[id][0].setMax(d, ranges[oldID][0].getMax(d));
+          }
+          if (VERBOSE) {
+            System.out.println("  id=" + id + " box=" + ranges[id] + " (same box as doc=" + oldID + ")");
+          }
+        } else {
+          for (int m = 0, even = dimensions % 2; m < dimensions * 2; ++m) {
+            if (x == m) {
+              int d = (int)Math.floor(m/2);
+              // current could be multivalue but old may not be, so use first box
+              if (even == 0) {
+                ranges[id][0].setMin(d, ranges[oldID][0].getMin(d));
+                if (VERBOSE) {
+                  System.out.println("  id=" + id + " box=" + ranges[id] + " (same min[" + d + "] as doc=" + oldID + ")");
+                }
+              } else {
+                ranges[id][0].setMax(d, ranges[oldID][0].getMax(d));
+                if (VERBOSE) {
+                  System.out.println("  id=" + id + " box=" + ranges[id] + " (same max[" + d + "] as doc=" + oldID + ")");
+                }
+              }
+            }
+          }
+        }
+      }
+    }
+    verify(ranges);
+  }
+
+  private void verify(Range[][] ranges) throws Exception {
+    IndexWriterConfig iwc = newIndexWriterConfig();
+    // Else seeds may not reproduce:
+    iwc.setMergeScheduler(new SerialMergeScheduler());
+    // Else we can get O(N^2) merging
+    int mbd = iwc.getMaxBufferedDocs();
+    if (mbd != -1 && mbd < ranges.length/100) {
+      iwc.setMaxBufferedDocs(ranges.length/100);
+    }
+    Directory dir;
+    if (ranges.length > 50000) {
+      dir = newFSDirectory(createTempDir(getClass().getSimpleName()));
+    } else {
+      dir = newDirectory();
+    }
+
+    Set<Integer> deleted = new HashSet<>();
+    IndexWriter w = new IndexWriter(dir, iwc);
+    for (int id=0; id < ranges.length; ++id) {
+      Document doc = new Document();
+      doc.add(newStringField("id", ""+id, Field.Store.NO));
+      doc.add(new NumericDocValuesField("id", id));
+      if (ranges[id][0].isMissing == false) {
+        for (int n=0; n<ranges[id].length; ++n) {
+          doc.add(newRangeField(ranges[id][n]));
+        }
+      }
+      w.addDocument(doc);
+      if (id > 0 && random().nextInt(100) == 1) {
+        int idToDelete = random().nextInt(id);
+        w.deleteDocuments(new Term("id", ""+idToDelete));
+        deleted.add(idToDelete);
+        if (VERBOSE) {
+          System.out.println("  delete id=" + idToDelete);
+        }
+      }
+    }
+
+    if (random().nextBoolean()) {
+      w.forceMerge(1);
+    }
+    final IndexReader r = DirectoryReader.open(w);
+    w.close();
+    IndexSearcher s = newSearcher(r);
+
+    int dimensions = ranges[0][0].numDimensions();
+    int iters = atLeast(25);
+    Bits liveDocs = MultiFields.getLiveDocs(s.getIndexReader());
+    int maxDoc = s.getIndexReader().maxDoc();
+
+    for (int iter=0; iter<iters; ++iter) {
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter + " s=" + s);
+      }
+
+      // occasionally test open ended bounding ranges
+      Range queryRange = nextRange(dimensions);
+      int rv = random().nextInt(4);
+      Query query;
+      Range.QueryType queryType;
+      if (rv == 0) {
+        queryType = Range.QueryType.INTERSECTS;
+        query = newIntersectsQuery(queryRange);
+      } else if (rv == 1)  {
+        queryType = Range.QueryType.CONTAINS;
+        query = newContainsQuery(queryRange);
+      } else if (rv == 2) {
+        queryType = Range.QueryType.WITHIN;
+        query = newWithinQuery(queryRange);
+      } else {
+        queryType = Range.QueryType.CROSSES;
+        query = newCrossesQuery(queryRange);
+      }
+
+      if (VERBOSE) {
+        System.out.println("  query=" + query);
+      }
+
+      final FixedBitSet hits = new FixedBitSet(maxDoc);
+      s.search(query, new SimpleCollector() {
+        private int docBase;
+
+        @Override
+        public void collect(int doc) {
+          hits.set(docBase + doc);
+        }
+
+        @Override
+        protected void doSetNextReader(LeafReaderContext context) throws IOException {
+          docBase = context.docBase;
+        }
+
+        @Override
+        public boolean needsScores() { return false; }
+      });
+
+      NumericDocValues docIDToID = MultiDocValues.getNumericValues(r, "id");
+      for (int docID=0; docID<maxDoc; ++docID) {
+        assertEquals(docID, docIDToID.nextDoc());
+        int id = (int) docIDToID.longValue();
+        boolean expected;
+        if (liveDocs != null && liveDocs.get(docID) == false) {
+          // document is deleted
+          expected = false;
+        } else if (ranges[id][0].isMissing) {
+          expected = false;
+        } else {
+          expected = expectedResult(queryRange, ranges[id], queryType);
+        }
+
+        if (hits.get(docID) != expected) {
+          StringBuilder b = new StringBuilder();
+          b.append("FAIL (iter " + iter + "): ");
+          if (expected == true) {
+            b.append("id=" + id + (ranges[id].length > 1 ? " (MultiValue) " : " ") + "should match but did not\n");
+          } else {
+            b.append("id=" + id + " should not match but did\n");
+          }
+          b.append(" queryRange=" + queryRange + "\n");
+          b.append(" box" + ((ranges[id].length > 1) ? "es=" : "=" ) + ranges[id][0]);
+          for (int n=1; n<ranges[id].length; ++n) {
+            b.append(", ");
+            b.append(ranges[id][n]);
+          }
+          b.append("\n queryType=" + queryType + "\n");
+          b.append(" deleted?=" + (liveDocs != null && liveDocs.get(docID) == false));
+          fail("wrong hit (first of possibly more):\n\n" + b);
+        }
+      }
+    }
+    IOUtils.close(r, dir);
+  }
+
+  protected boolean expectedResult(Range queryRange, Range[] range, Range.QueryType queryType) {
+    for (int i=0; i<range.length; ++i) {
+      if (expectedBBoxQueryResult(queryRange, range[i], queryType) == true) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  protected boolean expectedBBoxQueryResult(Range queryRange, Range range, Range.QueryType queryType) {
+    if (queryRange.isEqual(range) && queryType != Range.QueryType.CROSSES) {
+      return true;
+    }
+    Range.QueryType relation = range.relate(queryRange);
+    if (queryType == Range.QueryType.INTERSECTS) {
+      return relation != null;
+    } else if (queryType == Range.QueryType.CROSSES) {
+      // by definition, RangeFields that CONTAIN the query are also considered to cross
+      return relation == queryType || relation == Range.QueryType.CONTAINS;
+    }
+    return relation == queryType;
+  }
+
+  /** base class for range verification */
+  protected abstract static class Range {
+    protected boolean isMissing = false;
+
+    /** supported query relations */
+    protected enum QueryType { INTERSECTS, WITHIN, CONTAINS, CROSSES }
+
+    protected abstract int numDimensions();
+    protected abstract Object getMin(int dim);
+    protected abstract void setMin(int dim, Object val);
+    protected abstract Object getMax(int dim);
+    protected abstract void setMax(int dim, Object val);
+    protected abstract boolean isEqual(Range other);
+    protected abstract boolean isDisjoint(Range other);
+    protected abstract boolean isWithin(Range other);
+    protected abstract boolean contains(Range other);
+
+    protected QueryType relate(Range other) {
+      if (isDisjoint(other)) {
+        // if disjoint; return null:
+        return null;
+      } else if (isWithin(other)) {
+        return QueryType.WITHIN;
+      } else if (contains(other)) {
+        return QueryType.CONTAINS;
+      }
+      return QueryType.CROSSES;
+    }
+  }
+}


[05/43] lucene-solr:feature/autoscaling: LUCENE-7712: SimpleQueryParser now parses foo~ as foo~2

Posted by sh...@apache.org.
LUCENE-7712: SimpleQueryParser now parses foo~ as foo~2


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

Branch: refs/heads/feature/autoscaling
Commit: 21559fe86da5e84c75c25b8373f6c78f1ac75a8f
Parents: 7af6cc9
Author: Mike McCandless <mi...@apache.org>
Authored: Tue Mar 7 06:55:47 2017 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Tue Mar 7 06:55:47 2017 -0500

----------------------------------------------------------------------
 lucene/CHANGES.txt                                           | 4 ++++
 .../apache/lucene/queryparser/simple/SimpleQueryParser.java  | 8 +++++++-
 .../lucene/queryparser/simple/TestSimpleQueryParser.java     | 2 +-
 3 files changed, 12 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/21559fe8/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index a339760..b067fde 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -150,6 +150,10 @@ New Features
   efficiently filter out duplicate suggestions (Uwe Schindler, Mike
   McCandless)
 
+* LUCENE-7712: SimpleQueryParser now supports default fuzziness
+  syntax, mapping foo~ to a FuzzyQuery with edit distance 2.  (Lee
+  Hinman, David Pilato via Mike McCandless)
+
 Bug Fixes
 
 * LUCENE-7630: Fix (Edge)NGramTokenFilter to no longer drop payloads

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/21559fe8/lucene/queryparser/src/java/org/apache/lucene/queryparser/simple/SimpleQueryParser.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/simple/SimpleQueryParser.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/simple/SimpleQueryParser.java
index 6e05aec..2a7f7ea 100644
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/simple/SimpleQueryParser.java
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/simple/SimpleQueryParser.java
@@ -498,7 +498,13 @@ public class SimpleQueryParser extends QueryBuilder {
       }
       int fuzziness = 0;
       try {
-        fuzziness = Integer.parseInt(new String(slopText, 0, slopLength));
+        String fuzzyString =  new String(slopText, 0, slopLength);
+        if ("".equals(fuzzyString)) {
+          // Use automatic fuzziness, ~2
+          fuzziness = 2;
+        } else {
+          fuzziness = Integer.parseInt(fuzzyString);
+        }
       } catch (NumberFormatException e) {
         // swallow number format exceptions parsing fuzziness
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/21559fe8/lucene/queryparser/src/test/org/apache/lucene/queryparser/simple/TestSimpleQueryParser.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/simple/TestSimpleQueryParser.java b/lucene/queryparser/src/test/org/apache/lucene/queryparser/simple/TestSimpleQueryParser.java
index d70a22c..169caa2 100644
--- a/lucene/queryparser/src/test/org/apache/lucene/queryparser/simple/TestSimpleQueryParser.java
+++ b/lucene/queryparser/src/test/org/apache/lucene/queryparser/simple/TestSimpleQueryParser.java
@@ -88,7 +88,7 @@ public class TestSimpleQueryParser extends LuceneTestCase {
     Query expected = new FuzzyQuery(new Term("field", "foobar"), 2);
 
     assertEquals(expected, parse("foobar~2"));
-    assertEquals(regular, parse("foobar~"));
+    assertEquals(expected, parse("foobar~"));
     assertEquals(regular, parse("foobar~a"));
     assertEquals(regular, parse("foobar~1a"));
 


[02/43] lucene-solr:feature/autoscaling: SOLR-9986: Implement DatePointField

Posted by sh...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/core/src/test/org/apache/solr/search/TestSolrQueryParser.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestSolrQueryParser.java b/solr/core/src/test/org/apache/solr/search/TestSolrQueryParser.java
index 5c8e361..8195c05 100644
--- a/solr/core/src/test/org/apache/solr/search/TestSolrQueryParser.java
+++ b/solr/core/src/test/org/apache/solr/search/TestSolrQueryParser.java
@@ -122,9 +122,15 @@ public class TestSolrQueryParser extends SolrTestCaseJ4 {
     );
 
     // length of date math caused issues...
-    assertJQ(req("q", "foo_dt:\"2013-03-08T00:46:15Z/DAY+000MILLISECONDS+00SECONDS+00MINUTES+00HOURS+0000000000YEARS+6MONTHS+3DAYS\"", "debug", "query")
-        , "/debug/parsedquery=='foo_dt:2013-09-11T00:00:00Z'"
-    );
+    if (h.getCore().getLatestSchema().getField("foo_dt").getType().isPointField()) {
+      assertJQ(req("q", "foo_dt:\"2013-03-08T00:46:15Z/DAY+000MILLISECONDS+00SECONDS+00MINUTES+00HOURS+0000000000YEARS+6MONTHS+3DAYS\"", "debug", "query")
+          , "/debug/parsedquery=='IndexOrDocValuesQuery(foo_dt:[1378857600000 TO 1378857600000])'"
+      );
+    } else {
+      assertJQ(req("q", "foo_dt:\"2013-03-08T00:46:15Z/DAY+000MILLISECONDS+00SECONDS+00MINUTES+00HOURS+0000000000YEARS+6MONTHS+3DAYS\"", "debug", "query")
+          , "/debug/parsedquery=='foo_dt:2013-09-11T00:00:00Z'"
+      );
+    }
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/core/src/test/org/apache/solr/update/processor/AtomicUpdatesTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/processor/AtomicUpdatesTest.java b/solr/core/src/test/org/apache/solr/update/processor/AtomicUpdatesTest.java
index 28fdab3..7bae2c9 100644
--- a/solr/core/src/test/org/apache/solr/update/processor/AtomicUpdatesTest.java
+++ b/solr/core/src/test/org/apache/solr/update/processor/AtomicUpdatesTest.java
@@ -552,7 +552,12 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
 
     assertU(commit());
 
-    assertQ(req("q", "dateRemove:*", "indent", "true"), "//result[@numFound = '4']");
+    boolean isPointField = h.getCore().getLatestSchema().getField("dateRemove").getType().isPointField();
+    if (isPointField) {
+      assertQ(req("q", "dateRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
+    } else {
+      assertQ(req("q", "dateRemove:*", "indent", "true"), "//result[@numFound = '4']");
+    }
     assertQ(req("q", "dateRemove:\"2014-09-02T12:00:00Z\"", "indent", "true"), "//result[@numFound = '3']");
 
     doc = new SolrInputDocument();
@@ -565,7 +570,11 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
     assertU(adoc(doc));
     assertU(commit());
 
-    assertQ(req("q", "dateRemove:*", "indent", "true"), "//result[@numFound = '4']");
+    if (isPointField) {
+      assertQ(req("q", "dateRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
+    } else {
+      assertQ(req("q", "dateRemove:*", "indent", "true"), "//result[@numFound = '4']");
+    }
     assertQ(req("q", "dateRemove:\"2014-09-02T12:00:00Z\"", "indent", "true"), "//result[@numFound = '2']");
 
     doc = new SolrInputDocument();
@@ -577,7 +586,11 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
     assertU(adoc(doc));
     assertU(commit());
 
-    assertQ(req("q", "dateRemove:*", "indent", "true"), "//result[@numFound = '4']");
+    if (isPointField) {
+      assertQ(req("q", "dateRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
+    } else {
+      assertQ(req("q", "dateRemove:*", "indent", "true"), "//result[@numFound = '4']");
+    }
     assertQ(req("q", "dateRemove:\"2014-09-02T12:00:00Z\"", "indent", "true"), "//result[@numFound = '1']");
 
     doc = new SolrInputDocument();
@@ -587,7 +600,11 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
     assertU(adoc(doc));
     assertU(commit());
 
-    assertQ(req("q", "dateRemove:*", "indent", "true"), "//result[@numFound = '4']");
+    if (isPointField) {
+      assertQ(req("q", "dateRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
+    } else {
+      assertQ(req("q", "dateRemove:*", "indent", "true"), "//result[@numFound = '4']");
+    }
     assertQ(req("q", "dateRemove:\"2014-09-01T12:00:00Z\"", "indent", "true"), "//result[@numFound = '3']");
   }
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/core/src/test/org/apache/solr/update/processor/ParsingFieldUpdateProcessorsTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/processor/ParsingFieldUpdateProcessorsTest.java b/solr/core/src/test/org/apache/solr/update/processor/ParsingFieldUpdateProcessorsTest.java
index b779f7a..31f4760 100644
--- a/solr/core/src/test/org/apache/solr/update/processor/ParsingFieldUpdateProcessorsTest.java
+++ b/solr/core/src/test/org/apache/solr/update/processor/ParsingFieldUpdateProcessorsTest.java
@@ -18,6 +18,7 @@ package org.apache.solr.update.processor;
 
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.schema.PointField;
 import org.joda.time.DateTime;
 import org.joda.time.DateTimeZone;
 import org.joda.time.format.DateTimeFormat;
@@ -148,9 +149,15 @@ public class ParsingFieldUpdateProcessorsTest extends UpdateProcessorTestBase {
     String dateString = "2010-11-12T13:14:15.168Z";
     DateTimeFormatter dateTimeFormatter = ISODateTimeFormat.dateTime();
     DateTime dateTime = dateTimeFormatter.parseDateTime(dateString);
+    SolrInputDocument d;
+    if (schema.getField("date_dt").getType().isPointField()) {
+      d = processAdd("parse-date-explicit-typeclass-point-selector-no-run-processor",
+          doc(f("id", "77"), f("date_dt", dateString)));
+    } else {
+      d = processAdd("parse-date-explicit-typeclass-selector-no-run-processor",
+          doc(f("id", "77"), f("date_dt", dateString)));
+    }
 
-    SolrInputDocument d = processAdd("parse-date-explicit-typeclass-selector-no-run-processor",
-                                     doc(f("id", "77"), f("date_dt", dateString)));
     assertNotNull(d);
     assertTrue(d.getFieldValue("date_dt") instanceof Date);
     assertEquals(dateTime.getMillis(), ((Date)d.getFieldValue("date_dt")).getTime());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/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 c3c269c..a8c93d6 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
@@ -515,12 +515,14 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
       System.setProperty("solr.tests.longClass", "long");
       System.setProperty("solr.tests.doubleClass", "double");
       System.setProperty("solr.tests.floatClass", "float");
+      System.setProperty("solr.tests.dateClass", "date");
     } else {
       log.info("Using PointFields");
       System.setProperty("solr.tests.intClass", "pint");
       System.setProperty("solr.tests.longClass", "plong");
       System.setProperty("solr.tests.doubleClass", "pdouble");
       System.setProperty("solr.tests.floatClass", "pfloat");
+      System.setProperty("solr.tests.dateClass", "pdate");
     }
   }
 


[18/43] lucene-solr:feature/autoscaling: SOLR-8876: change morphline test config files to work around 'importCommands' bug when using java9

Posted by sh...@apache.org.
SOLR-8876: change morphline test config files to work around 'importCommands' bug when using java9


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

Branch: refs/heads/feature/autoscaling
Commit: 8756be05404758155b850748f807245fdaab6a8b
Parents: e35881a
Author: Chris Hostetter <ho...@apache.org>
Authored: Wed Mar 8 09:52:25 2017 -0700
Committer: Chris Hostetter <ho...@apache.org>
Committed: Wed Mar 8 09:52:46 2017 -0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  2 ++
 .../test/org/apache/solr/hadoop/MRUnitBase.java |  2 --
 .../MapReduceIndexerToolArgumentParserTest.java |  1 -
 .../solr/hadoop/MorphlineBasicMiniMRTest.java   |  1 -
 .../morphlines/cell/SolrCellMorphlineTest.java  |  2 --
 .../test-morphlines/loadSolrBasic.conf          |  7 +++++-
 .../test-morphlines/solrCellDocumentTypes.conf  | 23 +++++++++++++++++++-
 .../test-morphlines/solrCellJPGCompressed.conf  | 17 ++++++++++++++-
 .../test-files/test-morphlines/solrCellXML.conf | 11 +++++++++-
 .../test-morphlines/tokenizeText.conf           |  6 ++++-
 .../tutorialReadAvroContainer.conf              | 11 +++++++---
 .../solr/AbstractSolrMorphlineTestBase.java     |  2 --
 .../solr/AbstractSolrMorphlineZkTestBase.java   |  4 ----
 13 files changed, 69 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8756be05/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 0e78535..7285e4f 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -310,6 +310,8 @@ Other Changes
   correct now if other drivers were installed before test execution (e.g., through IDE).
   (hossman, Uwe Schindler)
 
+* SOLR-8876: change morphline test config files to work around 'importCommands' bug when using java9 (hossman)
+
 ==================  6.4.2 ==================
 
 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/8756be05/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MRUnitBase.java
----------------------------------------------------------------------
diff --git a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MRUnitBase.java b/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MRUnitBase.java
index 73323b9..558d662 100644
--- a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MRUnitBase.java
+++ b/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MRUnitBase.java
@@ -23,7 +23,6 @@ import java.util.Locale;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.lucene.util.Constants;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.hadoop.morphline.MorphlineMapRunner;
 import org.apache.solr.morphlines.solr.AbstractSolrMorphlineTestBase;
@@ -38,7 +37,6 @@ public abstract class MRUnitBase extends SolrTestCaseJ4 {
 
   @BeforeClass
   public static void setupClass() throws Exception {
-    assumeFalse("This test fails on Java 9 (https://issues.apache.org/jira/browse/SOLR-8876)", Constants.JRE_IS_MINIMUM_JAVA9);
     assumeFalse("This test fails on UNIX with Turkish default locale (https://issues.apache.org/jira/browse/SOLR-6387)",
         new Locale("tr").getLanguage().equals(Locale.getDefault().getLanguage()));
     solrHomeZip = SolrOutputFormat.createSolrHomeZip(new File(RESOURCES_DIR + "/solr/mrunit"));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8756be05/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MapReduceIndexerToolArgumentParserTest.java
----------------------------------------------------------------------
diff --git a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MapReduceIndexerToolArgumentParserTest.java b/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MapReduceIndexerToolArgumentParserTest.java
index 5dfb5cc..1aebcf7 100644
--- a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MapReduceIndexerToolArgumentParserTest.java
+++ b/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MapReduceIndexerToolArgumentParserTest.java
@@ -57,7 +57,6 @@ public class MapReduceIndexerToolArgumentParserTest extends SolrTestCaseJ4 {
   
   @BeforeClass
   public static void beforeClass() {
-    assumeFalse("This test fails on Java 9 (https://issues.apache.org/jira/browse/SOLR-8876)", Constants.JRE_IS_MINIMUM_JAVA9);
     assumeFalse("Does not work on Windows, because it uses UNIX shell commands or POSIX paths", Constants.WINDOWS);
     assumeFalse("This test fails on UNIX with Turkish default locale (https://issues.apache.org/jira/browse/SOLR-6387)",
                 new Locale("tr").getLanguage().equals(Locale.getDefault().getLanguage()));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8756be05/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineBasicMiniMRTest.java
----------------------------------------------------------------------
diff --git a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineBasicMiniMRTest.java b/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineBasicMiniMRTest.java
index b32f112..6479a20 100644
--- a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineBasicMiniMRTest.java
+++ b/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineBasicMiniMRTest.java
@@ -121,7 +121,6 @@ public class MorphlineBasicMiniMRTest extends SolrTestCaseJ4 {
     assumeFalse("HDFS tests were disabled by -Dtests.disableHdfs",
         Boolean.parseBoolean(System.getProperty("tests.disableHdfs", "false")));
     
-    assumeFalse("This test fails on Java 9 (https://issues.apache.org/jira/browse/SOLR-8876)", Constants.JRE_IS_MINIMUM_JAVA9);
     assumeFalse("FIXME: This test does not work with Windows because of native library requirements", Constants.WINDOWS);
     
     AbstractZkTestCase.SOLRHOME = solrHomeDirectory;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8756be05/solr/contrib/morphlines-cell/src/test/org/apache/solr/morphlines/cell/SolrCellMorphlineTest.java
----------------------------------------------------------------------
diff --git a/solr/contrib/morphlines-cell/src/test/org/apache/solr/morphlines/cell/SolrCellMorphlineTest.java b/solr/contrib/morphlines-cell/src/test/org/apache/solr/morphlines/cell/SolrCellMorphlineTest.java
index 3607252..e0872b6 100644
--- a/solr/contrib/morphlines-cell/src/test/org/apache/solr/morphlines/cell/SolrCellMorphlineTest.java
+++ b/solr/contrib/morphlines-cell/src/test/org/apache/solr/morphlines/cell/SolrCellMorphlineTest.java
@@ -42,8 +42,6 @@ public class SolrCellMorphlineTest extends AbstractSolrMorphlineTestBase {
   @BeforeClass
   public static void beforeClass2() {
     assumeFalse("FIXME: Morphlines currently has issues with Windows paths", Constants.WINDOWS);
-    assumeFalse("This test fails with Java 9 (https://issues.apache.org/jira/browse/SOLR-8876)",
-        Constants.JRE_IS_MINIMUM_JAVA9);
   }
 
   @Before

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8756be05/solr/contrib/morphlines-core/src/test-files/test-morphlines/loadSolrBasic.conf
----------------------------------------------------------------------
diff --git a/solr/contrib/morphlines-core/src/test-files/test-morphlines/loadSolrBasic.conf b/solr/contrib/morphlines-core/src/test-files/test-morphlines/loadSolrBasic.conf
index 190d0e4..1c02a9a 100644
--- a/solr/contrib/morphlines-core/src/test-files/test-morphlines/loadSolrBasic.conf
+++ b/solr/contrib/morphlines-core/src/test-files/test-morphlines/loadSolrBasic.conf
@@ -39,7 +39,12 @@ SOLR_LOCATOR : ${?ENV_SOLR_LOCATOR}
 morphlines : [
   {
     id : morphline1
-    importCommands : ["org.kitesdk.**", "org.apache.solr.**"]
+    
+    # using globs (foo.bar.* or foo.bar.**) will not work in Java9 due to classpath scanning limitations
+    # so we enumarate every command (builder) we know this config uses below. (see SOLR-8876)
+    importCommands : ["org.kitesdk.morphline.stdlib.LogDebugBuilder",
+                      "org.apache.solr.morphlines.solr.SanitizeUnknownSolrFieldsBuilder",
+		      "org.apache.solr.morphlines.solr.LoadSolrBuilder"]
     
     commands : [                    
       { 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8756be05/solr/contrib/morphlines-core/src/test-files/test-morphlines/solrCellDocumentTypes.conf
----------------------------------------------------------------------
diff --git a/solr/contrib/morphlines-core/src/test-files/test-morphlines/solrCellDocumentTypes.conf b/solr/contrib/morphlines-core/src/test-files/test-morphlines/solrCellDocumentTypes.conf
index 7d232dd..4d38256 100644
--- a/solr/contrib/morphlines-core/src/test-files/test-morphlines/solrCellDocumentTypes.conf
+++ b/solr/contrib/morphlines-core/src/test-files/test-morphlines/solrCellDocumentTypes.conf
@@ -22,7 +22,28 @@
 morphlines : [
   {
     id : morphline1
-    importCommands : ["org.kitesdk.**", "org.apache.solr.**"]
+
+    # using globs (foo.bar.* or foo.bar.**) will not work in Java9 due to classpath scanning limitations
+    # so we enumarate every command (builder) we know this config uses below. (see SOLR-8876)
+    importCommands : ["org.kitesdk.morphline.stdlib.LogDebugBuilder",
+                      "org.kitesdk.morphline.stdlib.SetValuesBuilder",
+                      "org.kitesdk.morphline.stdlib.CallParentPipeBuilder",
+                      "org.kitesdk.morphline.stdlib.GenerateUUIDBuilder",
+                      "org.kitesdk.morphline.stdlib.JavaBuilder",
+                      "org.kitesdk.morphline.stdlib.TryRulesBuilder",
+                      "org.kitesdk.morphline.stdlib.SeparateAttachmentsBuilder",
+                      "org.kitesdk.morphline.stdio.ReadCSVBuilder",
+                      "org.kitesdk.morphline.avro.ReadAvroContainerBuilder",
+                      "org.kitesdk.morphline.avro.ExtractAvroPathsBuilder",
+                      "org.kitesdk.morphline.avro.ExtractAvroTreeBuilder",
+                      "org.kitesdk.morphline.tika.DetectMimeTypeBuilder",
+                      "org.kitesdk.morphline.tika.decompress.DecompressBuilder",
+                      "org.kitesdk.morphline.tika.decompress.UnpackBuilder",
+                      "org.kitesdk.morphline.twitter.ReadJsonTestTweetsBuilder",
+                      "org.apache.solr.morphlines.cell.SolrCellBuilder",
+		      "org.apache.solr.morphlines.solr.SanitizeUnknownSolrFieldsBuilder",
+                      "org.apache.solr.morphlines.solr.GenerateSolrSequenceKeyBuilder",
+		      "org.apache.solr.morphlines.solr.LoadSolrBuilder"]
     
     commands : [
       { separateAttachments {} }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8756be05/solr/contrib/morphlines-core/src/test-files/test-morphlines/solrCellJPGCompressed.conf
----------------------------------------------------------------------
diff --git a/solr/contrib/morphlines-core/src/test-files/test-morphlines/solrCellJPGCompressed.conf b/solr/contrib/morphlines-core/src/test-files/test-morphlines/solrCellJPGCompressed.conf
index 66e7d40..85cb2a7 100644
--- a/solr/contrib/morphlines-core/src/test-files/test-morphlines/solrCellJPGCompressed.conf
+++ b/solr/contrib/morphlines-core/src/test-files/test-morphlines/solrCellJPGCompressed.conf
@@ -25,7 +25,22 @@
 morphlines : [
   {
     id : morphline1
-    importCommands : ["org.kitesdk.**", "org.apache.solr.**"]
+
+    # using globs (foo.bar.* or foo.bar.**) will not work in Java9 due to classpath scanning limitations
+    # so we enumarate every command (builder) we know this config uses below. (see SOLR-8876)
+    importCommands : ["org.kitesdk.morphline.stdlib.LogDebugBuilder",
+                      "org.kitesdk.morphline.stdlib.CallParentPipeBuilder",
+                      "org.kitesdk.morphline.stdlib.GenerateUUIDBuilder",
+                      "org.kitesdk.morphline.stdlib.JavaBuilder",
+                      "org.kitesdk.morphline.stdlib.TryRulesBuilder",
+                      "org.kitesdk.morphline.stdlib.SeparateAttachmentsBuilder",
+                      "org.kitesdk.morphline.tika.DetectMimeTypeBuilder",
+                      "org.kitesdk.morphline.tika.decompress.DecompressBuilder",
+                      "org.kitesdk.morphline.tika.decompress.UnpackBuilder",
+                      "org.apache.solr.morphlines.cell.SolrCellBuilder",
+                      "org.apache.solr.morphlines.solr.GenerateSolrSequenceKeyBuilder",
+		      "org.apache.solr.morphlines.solr.LoadSolrBuilder"]
+
     
     commands : [
       { separateAttachments {} }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8756be05/solr/contrib/morphlines-core/src/test-files/test-morphlines/solrCellXML.conf
----------------------------------------------------------------------
diff --git a/solr/contrib/morphlines-core/src/test-files/test-morphlines/solrCellXML.conf b/solr/contrib/morphlines-core/src/test-files/test-morphlines/solrCellXML.conf
index 43009bd..9e840a9 100644
--- a/solr/contrib/morphlines-core/src/test-files/test-morphlines/solrCellXML.conf
+++ b/solr/contrib/morphlines-core/src/test-files/test-morphlines/solrCellXML.conf
@@ -25,7 +25,16 @@
 morphlines : [
   {
     id : morphline1
-    importCommands : ["org.kitesdk.**", "org.apache.solr.**"]
+
+    # using globs (foo.bar.* or foo.bar.**) will not work in Java9 due to classpath scanning limitations
+    # so we enumarate every command (builder) we know this config uses below. (see SOLR-8876)
+    importCommands : ["org.kitesdk.morphline.stdlib.LogDebugBuilder",
+                      "org.kitesdk.morphline.stdlib.AddValuesBuilder",
+                      "org.apache.solr.morphlines.cell.SolrCellBuilder",
+                      "org.apache.solr.morphlines.solr.GenerateSolrSequenceKeyBuilder",
+                      "org.apache.solr.morphlines.solr.SanitizeUnknownSolrFieldsBuilder",
+		      "org.apache.solr.morphlines.solr.LoadSolrBuilder"]
+    
     
     commands : [
       {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8756be05/solr/contrib/morphlines-core/src/test-files/test-morphlines/tokenizeText.conf
----------------------------------------------------------------------
diff --git a/solr/contrib/morphlines-core/src/test-files/test-morphlines/tokenizeText.conf b/solr/contrib/morphlines-core/src/test-files/test-morphlines/tokenizeText.conf
index 9b62276..d9354c4 100644
--- a/solr/contrib/morphlines-core/src/test-files/test-morphlines/tokenizeText.conf
+++ b/solr/contrib/morphlines-core/src/test-files/test-morphlines/tokenizeText.conf
@@ -16,7 +16,11 @@
 morphlines : [
   {
     id : morphline1
-    importCommands : ["org.kitesdk.**", "org.apache.solr.**"]
+    
+    # using globs (foo.bar.* or foo.bar.**) will not work in Java9 due to classpath scanning limitations
+    # so we enumarate every command (builder) we know this config uses below. (see SOLR-8876)
+    importCommands : ["org.kitesdk.morphline.stdlib.LogDebugBuilder",
+                      "org.apache.solr.morphlines.solr.TokenizeTextBuilder"]
     
     commands : [                    
       { 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8756be05/solr/contrib/morphlines-core/src/test-files/test-morphlines/tutorialReadAvroContainer.conf
----------------------------------------------------------------------
diff --git a/solr/contrib/morphlines-core/src/test-files/test-morphlines/tutorialReadAvroContainer.conf b/solr/contrib/morphlines-core/src/test-files/test-morphlines/tutorialReadAvroContainer.conf
index 0c00686..eee4ba5 100644
--- a/solr/contrib/morphlines-core/src/test-files/test-morphlines/tutorialReadAvroContainer.conf
+++ b/solr/contrib/morphlines-core/src/test-files/test-morphlines/tutorialReadAvroContainer.conf
@@ -42,9 +42,14 @@ morphlines : [
     # morphline config file
     id : morphline1 
     
-    # Import all morphline commands in these java packages and their subpackages.
-    # Other commands that may be present on the classpath are not visible to this morphline.
-    importCommands : ["org.kitesdk.**", "org.apache.solr.**"]
+    # using globs (foo.bar.* or foo.bar.**) will not work in Java9 due to classpath scanning limitations
+    # so we enumarate every command (builder) we know this config uses below. (see SOLR-8876)
+    importCommands : ["org.kitesdk.morphline.stdlib.LogDebugBuilder",
+                      "org.kitesdk.morphline.avro.ReadAvroContainerBuilder",
+                      "org.kitesdk.morphline.avro.ExtractAvroPathsBuilder",
+                      "org.kitesdk.morphline.stdlib.ConvertTimestampBuilder",
+                      "org.apache.solr.morphlines.solr.SanitizeUnknownSolrFieldsBuilder",
+		      "org.apache.solr.morphlines.solr.LoadSolrBuilder"]
     
     commands : [                    
       { 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8756be05/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/AbstractSolrMorphlineTestBase.java
----------------------------------------------------------------------
diff --git a/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/AbstractSolrMorphlineTestBase.java b/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/AbstractSolrMorphlineTestBase.java
index 9a5791e..c91f31b 100644
--- a/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/AbstractSolrMorphlineTestBase.java
+++ b/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/AbstractSolrMorphlineTestBase.java
@@ -20,7 +20,6 @@ import com.codahale.metrics.MetricRegistry;
 import com.google.common.io.Files;
 import com.typesafe.config.Config;
 import org.apache.commons.io.FileUtils;
-import org.apache.lucene.util.Constants;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrQuery;
@@ -85,7 +84,6 @@ public class AbstractSolrMorphlineTestBase extends SolrTestCaseJ4 {
 
   @BeforeClass
   public static void beforeClass() throws Exception {
-    assumeFalse("This test fails on Java 9 (https://issues.apache.org/jira/browse/SOLR-8876)", Constants.JRE_IS_MINIMUM_JAVA9);
     
     // TODO: test doesn't work with some Locales, see SOLR-6458
     savedLocale = Locale.getDefault();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8756be05/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/AbstractSolrMorphlineZkTestBase.java
----------------------------------------------------------------------
diff --git a/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/AbstractSolrMorphlineZkTestBase.java b/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/AbstractSolrMorphlineZkTestBase.java
index 9aa27c4..31e7ebf 100644
--- a/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/AbstractSolrMorphlineZkTestBase.java
+++ b/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/AbstractSolrMorphlineZkTestBase.java
@@ -24,7 +24,6 @@ import java.util.Locale;
 import com.codahale.metrics.MetricRegistry;
 import com.google.common.collect.ListMultimap;
 import com.typesafe.config.Config;
-import org.apache.lucene.util.Constants;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.cloud.AbstractDistribZkTestBase;
@@ -79,9 +78,6 @@ public abstract class AbstractSolrMorphlineZkTestBase extends SolrCloudTestCase
   @BeforeClass
   public static void setupClass() throws Exception {
 
-    assumeFalse("This test fails on Java 9 (https://issues.apache.org/jira/browse/SOLR-8876)",
-        Constants.JRE_IS_MINIMUM_JAVA9);
-    
     assumeFalse("This test fails on UNIX with Turkish default locale (https://issues.apache.org/jira/browse/SOLR-6387)",
         new Locale("tr").getLanguage().equals(Locale.getDefault().getLanguage()));
 


[13/43] lucene-solr:feature/autoscaling: SOLR-9858: Collect aggregated metrics from nodes and shard leaders in overseer.

Posted by sh...@apache.org.
SOLR-9858: Collect aggregated metrics from nodes and shard leaders in overseer.


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

Branch: refs/heads/feature/autoscaling
Commit: 4d7bc9477144937335e997ad630c4b89f558ddc5
Parents: a6e14ec
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Tue Mar 7 22:00:38 2017 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Tue Mar 7 22:01:21 2017 +0100

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   4 +
 .../org/apache/solr/cloud/ElectionContext.java  |   5 +-
 .../java/org/apache/solr/cloud/Overseer.java    |   7 +-
 .../solr/cloud/OverseerNodePrioritizer.java     |   2 +-
 .../solr/cloud/OverseerTaskProcessor.java       |   6 +-
 .../org/apache/solr/cloud/ZkController.java     |   2 +-
 .../org/apache/solr/core/CoreContainer.java     |  30 +-
 .../org/apache/solr/core/JmxMonitoredMap.java   |   9 +-
 .../src/java/org/apache/solr/core/SolrCore.java |   4 +-
 .../org/apache/solr/core/SolrInfoMBean.java     |   4 +-
 .../org/apache/solr/core/SolrXmlConfig.java     |   3 +-
 .../handler/admin/MetricsCollectorHandler.java  | 228 +++++++++++
 .../solr/handler/admin/MetricsHandler.java      |   2 +-
 .../apache/solr/metrics/AggregateMetric.java    | 200 ++++++++++
 .../solr/metrics/SolrCoreMetricManager.java     | 125 +++++-
 .../apache/solr/metrics/SolrMetricManager.java  | 325 ++++++++++++++-
 .../metrics/reporters/JmxObjectNameFactory.java |   6 +-
 .../reporters/solr/SolrClusterReporter.java     | 277 +++++++++++++
 .../metrics/reporters/solr/SolrReporter.java    | 392 +++++++++++++++++++
 .../reporters/solr/SolrShardReporter.java       | 188 +++++++++
 .../metrics/reporters/solr/package-info.java    |  22 ++
 .../java/org/apache/solr/update/PeerSync.java   |   8 +-
 .../org/apache/solr/util/stats/MetricUtils.java | 265 +++++++++----
 .../src/test-files/solr/solr-solrreporter.xml   |  66 ++++
 .../apache/solr/cloud/TestCloudRecovery.java    |   6 +-
 .../apache/solr/core/TestJmxMonitoredMap.java   |   2 +-
 .../solr/metrics/SolrCoreMetricManagerTest.java |  31 +-
 .../solr/metrics/SolrMetricManagerTest.java     |  30 +-
 .../metrics/SolrMetricsIntegrationTest.java     |  15 +-
 .../metrics/reporters/SolrJmxReporterTest.java  |  13 +-
 .../reporters/solr/SolrCloudReportersTest.java  | 163 ++++++++
 .../reporters/solr/SolrShardReporterTest.java   | 117 ++++++
 .../apache/solr/util/stats/MetricUtilsTest.java |  54 ++-
 .../client/solrj/impl/BinaryRequestWriter.java  |   4 +-
 .../solr/client/solrj/io/SolrClientCache.java   |  26 +-
 .../client/solrj/request/TestCoreAdmin.java     |   4 +-
 36 files changed, 2435 insertions(+), 210 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d7bc947/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index dc97456..0e78535 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -50,6 +50,10 @@ Upgrading from Solr 6.x
   factors should be indexed in a separate field and combined with the query
   score using a function query.
 
+New Features
+----------------------
+* SOLR-9857, SOLR-9858: Collect aggregated metrics from nodes and shard leaders in overseer. (ab)
+
 Bug Fixes
 ----------------------
 * SOLR-9262: Connection and read timeouts are being ignored by UpdateShardHandler after SOLR-4509.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d7bc947/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java b/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
index ff6fb30..d3ad322 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
@@ -714,14 +714,13 @@ final class OverseerElectionContext extends ElectionContext {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private final SolrZkClient zkClient;
   private Overseer overseer;
-  public static final String OVERSEER_ELECT = "/overseer_elect";
 
   public OverseerElectionContext(SolrZkClient zkClient, Overseer overseer, final String zkNodeName) {
-    super(zkNodeName, OVERSEER_ELECT, OVERSEER_ELECT + "/leader", null, zkClient);
+    super(zkNodeName, Overseer.OVERSEER_ELECT, Overseer.OVERSEER_ELECT + "/leader", null, zkClient);
     this.overseer = overseer;
     this.zkClient = zkClient;
     try {
-      new ZkCmdExecutor(zkClient.getZkClientTimeout()).ensureExists(OVERSEER_ELECT, zkClient);
+      new ZkCmdExecutor(zkClient.getZkClientTimeout()).ensureExists(Overseer.OVERSEER_ELECT, zkClient);
     } catch (KeeperException e) {
       throw new SolrException(ErrorCode.SERVER_ERROR, e);
     } catch (InterruptedException e) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d7bc947/solr/core/src/java/org/apache/solr/cloud/Overseer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/Overseer.java b/solr/core/src/java/org/apache/solr/cloud/Overseer.java
index 3a8aa3e..61f15fc 100644
--- a/solr/core/src/java/org/apache/solr/cloud/Overseer.java
+++ b/solr/core/src/java/org/apache/solr/cloud/Overseer.java
@@ -65,7 +65,8 @@ public class Overseer implements Closeable {
   public static final int STATE_UPDATE_DELAY = 1500;  // delay between cloud state updates
 
   public static final int NUM_RESPONSES_TO_STORE = 10000;
-  
+  public static final String OVERSEER_ELECT = "/overseer_elect";
+
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   enum LeaderStatus {DONT_KNOW, NO, YES}
@@ -281,7 +282,7 @@ public class Overseer implements Closeable {
     private void checkIfIamStillLeader() {
       if (zkController != null && zkController.getCoreContainer().isShutDown()) return;//shutting down no need to go further
       org.apache.zookeeper.data.Stat stat = new org.apache.zookeeper.data.Stat();
-      String path = OverseerElectionContext.OVERSEER_ELECT + "/leader";
+      String path = OVERSEER_ELECT + "/leader";
       byte[] data;
       try {
         data = zkClient.getData(path, null, stat, true);
@@ -394,7 +395,7 @@ public class Overseer implements Closeable {
       boolean success = true;
       try {
         ZkNodeProps props = ZkNodeProps.load(zkClient.getData(
-            OverseerElectionContext.OVERSEER_ELECT + "/leader", null, null, true));
+            OVERSEER_ELECT + "/leader", null, null, true));
         if (myId.equals(props.getStr("id"))) {
           return LeaderStatus.YES;
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d7bc947/solr/core/src/java/org/apache/solr/cloud/OverseerNodePrioritizer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerNodePrioritizer.java b/solr/core/src/java/org/apache/solr/cloud/OverseerNodePrioritizer.java
index 6512d26..798eca3 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerNodePrioritizer.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerNodePrioritizer.java
@@ -65,7 +65,7 @@ public class OverseerNodePrioritizer {
     String ldr = OverseerTaskProcessor.getLeaderNode(zk);
     if(overseerDesignates.contains(ldr)) return;
     log.info("prioritizing overseer nodes at {} overseer designates are {}", overseerId, overseerDesignates);
-    List<String> electionNodes = OverseerTaskProcessor.getSortedElectionNodes(zk, OverseerElectionContext.OVERSEER_ELECT + LeaderElector.ELECTION_NODE);
+    List<String> electionNodes = OverseerTaskProcessor.getSortedElectionNodes(zk, Overseer.OVERSEER_ELECT + LeaderElector.ELECTION_NODE);
     if(electionNodes.size()<2) return;
     log.info("sorted nodes {}", electionNodes);
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d7bc947/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java
index ad53346..bed71a6 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java
@@ -337,7 +337,7 @@ public class OverseerTaskProcessor implements Runnable, Closeable {
   public static List<String> getSortedOverseerNodeNames(SolrZkClient zk) throws KeeperException, InterruptedException {
     List<String> children = null;
     try {
-      children = zk.getChildren(OverseerElectionContext.OVERSEER_ELECT + LeaderElector.ELECTION_NODE, null, true);
+      children = zk.getChildren(Overseer.OVERSEER_ELECT + LeaderElector.ELECTION_NODE, null, true);
     } catch (Exception e) {
       log.warn("error ", e);
       return new ArrayList<>();
@@ -370,7 +370,7 @@ public class OverseerTaskProcessor implements Runnable, Closeable {
   public static String getLeaderId(SolrZkClient zkClient) throws KeeperException,InterruptedException{
     byte[] data = null;
     try {
-      data = zkClient.getData(OverseerElectionContext.OVERSEER_ELECT + "/leader", null, new Stat(), true);
+      data = zkClient.getData(Overseer.OVERSEER_ELECT + "/leader", null, new Stat(), true);
     } catch (KeeperException.NoNodeException e) {
       return null;
     }
@@ -384,7 +384,7 @@ public class OverseerTaskProcessor implements Runnable, Closeable {
     boolean success = true;
     try {
       ZkNodeProps props = ZkNodeProps.load(zkStateReader.getZkClient().getData(
-          OverseerElectionContext.OVERSEER_ELECT + "/leader", null, null, true));
+          Overseer.OVERSEER_ELECT + "/leader", null, null, true));
       if (myId.equals(props.getStr("id"))) {
         return LeaderStatus.YES;
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d7bc947/solr/core/src/java/org/apache/solr/cloud/ZkController.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkController.java b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
index c083736..333acd4 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
@@ -1715,7 +1715,7 @@ public class ZkController {
           //however delete it . This is possible when the last attempt at deleting the election node failed.
           if (electionNode.startsWith(getNodeName())) {
             try {
-              zkClient.delete(OverseerElectionContext.OVERSEER_ELECT + LeaderElector.ELECTION_NODE + "/" + electionNode, -1, true);
+              zkClient.delete(Overseer.OVERSEER_ELECT + LeaderElector.ELECTION_NODE + "/" + electionNode, -1, true);
             } catch (NoNodeException e) {
               //no problem
             } catch (InterruptedException e) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d7bc947/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 e3977d7..b9597ae 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -69,6 +69,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.MetricsCollectorHandler;
 import org.apache.solr.handler.admin.MetricsHandler;
 import org.apache.solr.handler.admin.SecurityConfHandler;
 import org.apache.solr.handler.admin.SecurityConfHandlerLocal;
@@ -177,6 +178,8 @@ public class CoreContainer {
 
   protected MetricsHandler metricsHandler;
 
+  protected MetricsCollectorHandler metricsCollectorHandler;
+
   private enum CoreInitFailedAction { fromleader, none }
 
   /**
@@ -511,15 +514,18 @@ public class CoreContainer {
     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);
+    metricsCollectorHandler = createHandler(MetricsCollectorHandler.HANDLER_PATH, MetricsCollectorHandler.class.getName(), MetricsCollectorHandler.class);
+    // may want to add some configuration here in the future
+    metricsCollectorHandler.init(null);
     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, null, SolrInfoMBean.Group.node);
+    metricManager.loadReporters(cfg.getMetricReporterPlugins(), loader, null, SolrInfoMBean.Group.jvm);
+    metricManager.loadReporters(cfg.getMetricReporterPlugins(), loader, null, SolrInfoMBean.Group.jetty);
 
     coreConfigService = ConfigSetService.createConfigSetService(cfg, loader, zkSys.zkController);
 
@@ -537,6 +543,10 @@ public class CoreContainer {
     metricManager.register(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node),
         unloadedCores, true, "unloaded",SolrInfoMBean.Category.CONTAINER.toString(), "cores");
 
+    if (isZooKeeperAware()) {
+      metricManager.loadClusterReporters(cfg.getMetricReporterPlugins(), this);
+    }
+
     // setup executor to load cores in parallel
     ExecutorService coreLoadExecutor = MetricUtils.instrumentedExecutorService(
         ExecutorUtil.newMDCAwareFixedThreadPool(
@@ -660,10 +670,16 @@ public class CoreContainer {
     isShutDown = true;
 
     ExecutorUtil.shutdownAndAwaitTermination(coreContainerWorkExecutor);
+    if (metricManager != null) {
+      metricManager.closeReporters(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node));
+    }
 
     if (isZooKeeperAware()) {
       cancelCoreRecoveries();
-      zkSys.zkController.publishNodeAsDown(zkSys.zkController.getNodeName()); 
+      zkSys.zkController.publishNodeAsDown(zkSys.zkController.getNodeName());
+      if (metricManager != null) {
+        metricManager.closeReporters(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.cluster));
+      }
     }
 
     try {
@@ -722,10 +738,6 @@ 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) {
@@ -1232,7 +1244,7 @@ public class CoreContainer {
     try (SolrCore core = getCore(name)) {
       if (core != null) {
         String oldRegistryName = core.getCoreMetricManager().getRegistryName();
-        String newRegistryName = SolrCoreMetricManager.createRegistryName(core.getCoreDescriptor().getCollectionName(), toName);
+        String newRegistryName = SolrCoreMetricManager.createRegistryName(core, toName);
         metricManager.swapRegistries(oldRegistryName, newRegistryName);
         registerCore(toName, core, true, false);
         SolrCore old = solrCores.remove(name);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d7bc947/solr/core/src/java/org/apache/solr/core/JmxMonitoredMap.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/JmxMonitoredMap.java b/solr/core/src/java/org/apache/solr/core/JmxMonitoredMap.java
index b2a5c79..8bfa662 100644
--- a/solr/core/src/java/org/apache/solr/core/JmxMonitoredMap.java
+++ b/solr/core/src/java/org/apache/solr/core/JmxMonitoredMap.java
@@ -20,6 +20,7 @@ import javax.management.Attribute;
 import javax.management.AttributeList;
 import javax.management.AttributeNotFoundException;
 import javax.management.DynamicMBean;
+import javax.management.InstanceNotFoundException;
 import javax.management.InvalidAttributeValueException;
 import javax.management.MBeanAttributeInfo;
 import javax.management.MBeanException;
@@ -53,7 +54,6 @@ import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.SolrConfig.JmxConfiguration;
-import org.apache.solr.metrics.SolrCoreMetricManager;
 import org.apache.solr.metrics.reporters.JmxObjectNameFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -93,9 +93,10 @@ public class JmxMonitoredMap<K, V> extends
 
   private final String registryName;
 
-  public JmxMonitoredMap(String coreName, String coreHashCode,
+  public JmxMonitoredMap(String coreName, String coreHashCode, String registryName,
                          final JmxConfiguration jmxConfig) {
     this.coreHashCode = coreHashCode;
+    this.registryName = registryName;
     jmxRootName = (null != jmxConfig.rootName ?
                    jmxConfig.rootName
                    : ("solr" + (null != coreName ? "/" + coreName : "")));
@@ -117,7 +118,6 @@ public class JmxMonitoredMap<K, V> extends
 
       if (servers == null || servers.isEmpty()) {
         server = null;
-        registryName = null;
         nameFactory = null;
         log.debug("No JMX servers found, not exposing Solr information with JMX.");
         return;
@@ -141,7 +141,6 @@ public class JmxMonitoredMap<K, V> extends
       }
       server = newServer;
     }
-    registryName = SolrCoreMetricManager.createRegistryName(null, coreName);
     nameFactory = new JmxObjectNameFactory(REPORTER_NAME + coreHashCode, registryName);
   }
 
@@ -166,6 +165,8 @@ public class JmxMonitoredMap<K, V> extends
         for (ObjectName name : objectNames) {
           try {
             server.unregisterMBean(name);
+          } catch (InstanceNotFoundException ie) {
+            // ignore - someone else already deleted this one
           } catch (Exception e) {
             log.warn("Exception un-registering mbean {}", name, e);
           }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d7bc947/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 f22c472..13c3bdd 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -860,6 +860,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
     this.configSetProperties = configSetProperties;
     // Initialize the metrics manager
     this.coreMetricManager = initCoreMetricManager(config);
+    this.coreMetricManager.loadReporters();
 
     if (updateHandler == null) {
       directoryFactory = initDirectoryFactory();
@@ -1101,13 +1102,12 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
    */
   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);
+      return new JmxMonitoredMap<String, SolrInfoMBean>(name, coreMetricManager.getRegistryName(), String.valueOf(this.hashCode()), config.jmxConfig);
     } else  {
       log.debug("JMX monitoring not detected for core: " + name);
       return new ConcurrentHashMap<>();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d7bc947/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 bf77db4..63bdef0 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java
@@ -36,9 +36,9 @@ public interface SolrInfoMBean {
     SEARCHER, REPLICATION, TLOG, INDEX, DIRECTORY, HTTP, OTHER }
 
   /**
-   * Top-level group of beans for a subsystem.
+   * Top-level group of beans or metrics for a subsystem.
    */
-  enum Group { jvm, jetty, node, core }
+  enum Group { jvm, jetty, node, core, collection, shard, cluster, overseer }
 
   /**
    * Simple common usage name, e.g. BasicQueryHandler,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d7bc947/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 e41cd8d..951d8d5 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
@@ -451,7 +451,8 @@ public class SolrXmlConfig {
       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);
+      // we don't require class in order to support predefined replica and node reporter classes
+      configs[i] = new PluginInfo(nodes.item(i), "SolrMetricReporter", true, false);
     }
     return configs;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d7bc947/solr/core/src/java/org/apache/solr/handler/admin/MetricsCollectorHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/MetricsCollectorHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/MetricsCollectorHandler.java
new file mode 100644
index 0000000..de39a61
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/admin/MetricsCollectorHandler.java
@@ -0,0 +1,228 @@
+/*
+ * 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.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.HashMap;
+import java.util.Map;
+
+import com.codahale.metrics.MetricRegistry;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.ContentStream;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.handler.loader.ContentStreamLoader;
+import org.apache.solr.handler.RequestHandlerBase;
+import org.apache.solr.handler.loader.CSVLoader;
+import org.apache.solr.handler.loader.JavabinLoader;
+import org.apache.solr.handler.loader.JsonLoader;
+import org.apache.solr.handler.loader.XMLLoader;
+import org.apache.solr.metrics.AggregateMetric;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.metrics.reporters.solr.SolrReporter;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.update.AddUpdateCommand;
+import org.apache.solr.update.CommitUpdateCommand;
+import org.apache.solr.update.DeleteUpdateCommand;
+import org.apache.solr.update.MergeIndexesCommand;
+import org.apache.solr.update.RollbackUpdateCommand;
+import org.apache.solr.update.processor.UpdateRequestProcessor;
+import org.apache.solr.util.stats.MetricUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Handler to collect and aggregate metric reports.  Each report indicates the target registry where
+ * metrics values should be collected and aggregated. Metrics with the same names are
+ * aggregated using {@link AggregateMetric} instances, which track the source of updates and
+ * their count, as well as providing simple statistics over collected values.
+ *
+ * Each report consists of {@link SolrInputDocument}-s that are expected to contain
+ * the following fields:
+ * <ul>
+ *   <li>{@link SolrReporter#GROUP_ID} - (required) specifies target registry name where metrics will be grouped.</li>
+ *   <li>{@link SolrReporter#REPORTER_ID} - (required) id of the reporter that sent this update. This can be eg.
+ *   node name or replica name or other id that uniquely identifies the source of metrics values.</li>
+ *   <li>{@link MetricUtils#METRIC_NAME} - (required) metric name (in the source registry)</li>
+ *   <li>{@link SolrReporter#LABEL_ID} - (optional) label to prepend to metric names in the target registry.</li>
+ *   <li>{@link SolrReporter#REGISTRY_ID} - (optional) name of the source registry.</li>
+ * </ul>
+ * Remaining fields are assumed to be single-valued, and to contain metric attributes and their values. Example:
+ * <pre>
+ *   &lt;doc&gt;
+ *     &lt;field name="_group_"&gt;solr.core.collection1.shard1.leader&lt;/field&gt;
+ *     &lt;field name="_reporter_"&gt;core_node3&lt;/field&gt;
+ *     &lt;field name="metric"&gt;INDEX.merge.errors&lt;/field&gt;
+ *     &lt;field name="value"&gt;0&lt;/field&gt;
+ *   &lt;/doc&gt;
+ * </pre>
+ */
+public class MetricsCollectorHandler extends RequestHandlerBase {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public static final String HANDLER_PATH = "/admin/metrics/collector";
+
+  private final CoreContainer coreContainer;
+  private final SolrMetricManager metricManager;
+  private final Map<String, ContentStreamLoader> loaders = new HashMap<>();
+  private SolrParams params;
+
+  public MetricsCollectorHandler(final CoreContainer coreContainer) {
+    this.coreContainer = coreContainer;
+    this.metricManager = coreContainer.getMetricManager();
+
+  }
+
+  @Override
+  public void init(NamedList initArgs) {
+    super.init(initArgs);
+    if (initArgs != null) {
+      params = SolrParams.toSolrParams(initArgs);
+    } else {
+      params = new ModifiableSolrParams();
+    }
+    loaders.put("application/xml", new XMLLoader().init(params) );
+    loaders.put("application/json", new JsonLoader().init(params) );
+    loaders.put("application/csv", new CSVLoader().init(params) );
+    loaders.put("application/javabin", new JavabinLoader().init(params) );
+    loaders.put("text/csv", loaders.get("application/csv") );
+    loaders.put("text/xml", loaders.get("application/xml") );
+    loaders.put("text/json", loaders.get("application/json"));
+  }
+
+  @Override
+  public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
+    if (coreContainer == null || coreContainer.isShutDown()) {
+      // silently drop request
+      return;
+    }
+    //log.info("#### " + req.toString());
+    if (req.getContentStreams() == null) { // no content
+      return;
+    }
+    for (ContentStream cs : req.getContentStreams()) {
+      if (cs.getContentType() == null) {
+        log.warn("Missing content type - ignoring");
+        continue;
+      }
+      ContentStreamLoader loader = loaders.get(cs.getContentType());
+      if (loader == null) {
+        throw new SolrException(SolrException.ErrorCode.UNSUPPORTED_MEDIA_TYPE, "Unsupported content type for stream: " + cs.getSourceInfo() + ", contentType=" + cs.getContentType());
+      }
+      loader.load(req, rsp, cs, new MetricUpdateProcessor(metricManager));
+    }
+  }
+
+  @Override
+  public String getDescription() {
+    return "Handler for collecting and aggregating metric reports.";
+  }
+
+  private static class MetricUpdateProcessor extends UpdateRequestProcessor {
+    private final SolrMetricManager metricManager;
+
+    public MetricUpdateProcessor(SolrMetricManager metricManager) {
+      super(null);
+      this.metricManager = metricManager;
+    }
+
+    @Override
+    public void processAdd(AddUpdateCommand cmd) throws IOException {
+      SolrInputDocument doc = cmd.solrDoc;
+      if (doc == null) {
+        return;
+      }
+      String metricName = (String)doc.getFieldValue(MetricUtils.METRIC_NAME);
+      if (metricName == null) {
+        log.warn("Missing " + MetricUtils.METRIC_NAME + " field in document, skipping: " + doc);
+        return;
+      }
+      doc.remove(MetricUtils.METRIC_NAME);
+      // XXX we could modify keys by using this original registry name
+      doc.remove(SolrReporter.REGISTRY_ID);
+      String groupId = (String)doc.getFieldValue(SolrReporter.GROUP_ID);
+      if (groupId == null) {
+        log.warn("Missing " + SolrReporter.GROUP_ID + " field in document, skipping: " + doc);
+        return;
+      }
+      doc.remove(SolrReporter.GROUP_ID);
+      String reporterId = (String)doc.getFieldValue(SolrReporter.REPORTER_ID);
+      if (reporterId == null) {
+        log.warn("Missing " + SolrReporter.REPORTER_ID + " field in document, skipping: " + doc);
+        return;
+      }
+      doc.remove(SolrReporter.REPORTER_ID);
+      String labelId = (String)doc.getFieldValue(SolrReporter.LABEL_ID);
+      doc.remove(SolrReporter.LABEL_ID);
+      doc.forEach(f -> {
+        String key = MetricRegistry.name(labelId, metricName, f.getName());
+        MetricRegistry registry = metricManager.registry(groupId);
+        AggregateMetric metric = getOrRegister(registry, key, new AggregateMetric());
+        Object o = f.getFirstValue();
+        if (o != null) {
+          metric.set(reporterId, o);
+        } else {
+          // remove missing values
+          metric.clear(reporterId);
+        }
+      });
+    }
+
+    private AggregateMetric getOrRegister(MetricRegistry registry, String name, AggregateMetric add) {
+      AggregateMetric existing = (AggregateMetric)registry.getMetrics().get(name);
+      if (existing != null) {
+        return existing;
+      }
+      try {
+        registry.register(name, add);
+        return add;
+      } catch (IllegalArgumentException e) {
+        // someone added before us
+        existing = (AggregateMetric)registry.getMetrics().get(name);
+        if (existing == null) { // now, that is weird...
+          throw new IllegalArgumentException("Inconsistent metric status, " + name);
+        }
+        return existing;
+      }
+    }
+
+    @Override
+    public void processDelete(DeleteUpdateCommand cmd) throws IOException {
+      throw new UnsupportedOperationException("processDelete");
+    }
+
+    @Override
+    public void processMergeIndexes(MergeIndexesCommand cmd) throws IOException {
+      throw new UnsupportedOperationException("processMergeIndexes");
+    }
+
+    @Override
+    public void processCommit(CommitUpdateCommand cmd) throws IOException {
+      throw new UnsupportedOperationException("processCommit");
+    }
+
+    @Override
+    public void processRollback(RollbackUpdateCommand cmd) throws IOException {
+      throw new UnsupportedOperationException("processRollback");
+    }
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d7bc947/solr/core/src/java/org/apache/solr/metrics/AggregateMetric.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/AggregateMetric.java b/solr/core/src/java/org/apache/solr/metrics/AggregateMetric.java
new file mode 100644
index 0000000..babc99d
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/metrics/AggregateMetric.java
@@ -0,0 +1,200 @@
+/*
+ * 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.Collections;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.codahale.metrics.Metric;
+
+/**
+ * This class is used for keeping several partial named values and providing useful statistics over them.
+ */
+public class AggregateMetric implements Metric {
+
+  /**
+   * Simple class to represent current value and how many times it was set.
+   */
+  public static class Update {
+    public Object value;
+    public final AtomicInteger updateCount = new AtomicInteger();
+
+    public Update(Object value) {
+      update(value);
+    }
+
+    public void update(Object value) {
+      this.value = value;
+      updateCount.incrementAndGet();
+    }
+
+    @Override
+    public String toString() {
+      return "Update{" +
+          "value=" + value +
+          ", updateCount=" + updateCount +
+          '}';
+    }
+  }
+
+  private final Map<String, Update> values = new ConcurrentHashMap<>();
+
+  public void set(String name, Object value) {
+    final Update existing = values.get(name);
+    if (existing == null) {
+      final Update created = new Update(value);
+      final Update raced = values.putIfAbsent(name, created);
+      if (raced != null) {
+        raced.update(value);
+      }
+    } else {
+      existing.update(value);
+    }
+  }
+
+  public void clear(String name) {
+    values.remove(name);
+  }
+
+  public void clear() {
+    values.clear();
+  }
+
+  public int size() {
+    return values.size();
+  }
+
+  public boolean isEmpty() {
+    return values.isEmpty();
+  }
+
+  public Map<String, Update> getValues() {
+    return Collections.unmodifiableMap(values);
+  }
+
+  // --------- stats ---------
+  public double getMax() {
+    if (values.isEmpty()) {
+      return 0;
+    }
+    Double res = null;
+    for (Update u : values.values()) {
+      if (!(u.value instanceof Number)) {
+        continue;
+      }
+      Number n = (Number)u.value;
+      if (res == null) {
+        res = n.doubleValue();
+        continue;
+      }
+      if (n.doubleValue() > res) {
+        res = n.doubleValue();
+      }
+    }
+    return res;
+  }
+
+  public double getMin() {
+    if (values.isEmpty()) {
+      return 0;
+    }
+    Double res = null;
+    for (Update u : values.values()) {
+      if (!(u.value instanceof Number)) {
+        continue;
+      }
+      Number n = (Number)u.value;
+      if (res == null) {
+        res = n.doubleValue();
+        continue;
+      }
+      if (n.doubleValue() < res) {
+        res = n.doubleValue();
+      }
+    }
+    return res;
+  }
+
+  public double getMean() {
+    if (values.isEmpty()) {
+      return 0;
+    }
+    double total = 0;
+    for (Update u : values.values()) {
+      if (!(u.value instanceof Number)) {
+        continue;
+      }
+      Number n = (Number)u.value;
+      total += n.doubleValue();
+    }
+    return total / values.size();
+  }
+
+  public double getStdDev() {
+    int size = values.size();
+    if (size < 2) {
+      return 0;
+    }
+    final double mean = getMean();
+    double sum = 0;
+    int count = 0;
+    for (Update u : values.values()) {
+      if (!(u.value instanceof Number)) {
+        continue;
+      }
+      count++;
+      Number n = (Number)u.value;
+      final double diff = n.doubleValue() - mean;
+      sum += diff * diff;
+    }
+    if (count < 2) {
+      return 0;
+    }
+    final double variance = sum / (count - 1);
+    return Math.sqrt(variance);
+  }
+
+  public double getSum() {
+    if (values.isEmpty()) {
+      return 0;
+    }
+    double res = 0;
+    for (Update u : values.values()) {
+      if (!(u.value instanceof Number)) {
+        continue;
+      }
+      Number n = (Number)u.value;
+      res += n.doubleValue();
+    }
+    return res;
+  }
+
+  @Override
+  public String toString() {
+    return "AggregateMetric{" +
+        "size=" + size() +
+        ", max=" + getMax() +
+        ", min=" + getMin() +
+        ", mean=" + getMean() +
+        ", stddev=" + getStdDev() +
+        ", sum=" + getSum() +
+        ", values=" + values +
+        '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d7bc947/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 eb5b687..43f3535 100644
--- a/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java
@@ -20,6 +20,7 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 
+import org.apache.solr.cloud.CloudDescriptor;
 import org.apache.solr.core.NodeConfig;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.SolrCore;
@@ -36,8 +37,14 @@ public class SolrCoreMetricManager implements Closeable {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   private final SolrCore core;
+  private final String tag;
   private final SolrMetricManager metricManager;
   private String registryName;
+  private String collectionName;
+  private String shardName;
+  private String replicaName;
+  private String leaderRegistryName;
+  private boolean cloudMode;
 
   /**
    * Constructs a metric manager.
@@ -46,8 +53,26 @@ public class SolrCoreMetricManager implements Closeable {
    */
   public SolrCoreMetricManager(SolrCore core) {
     this.core = core;
+    this.tag = String.valueOf(core.hashCode());
     this.metricManager = core.getCoreDescriptor().getCoreContainer().getMetricManager();
-    registryName = createRegistryName(core.getCoreDescriptor().getCollectionName(), core.getName());
+    initCloudMode();
+    registryName = createRegistryName(cloudMode, collectionName, shardName, replicaName, core.getName());
+    leaderRegistryName = createLeaderRegistryName(cloudMode, collectionName, shardName);
+  }
+
+  private void initCloudMode() {
+    CloudDescriptor cd = core.getCoreDescriptor().getCloudDescriptor();
+    if (cd != null) {
+      cloudMode = true;
+      collectionName = core.getCoreDescriptor().getCollectionName();
+      shardName = cd.getShardId();
+      //replicaName = cd.getCoreNodeName();
+      String coreName = core.getName();
+      replicaName = parseReplicaName(collectionName, coreName);
+      if (replicaName == null) {
+        replicaName = cd.getCoreNodeName();
+      }
+    }
   }
 
   /**
@@ -57,7 +82,11 @@ public class SolrCoreMetricManager implements Closeable {
   public void loadReporters() {
     NodeConfig nodeConfig = core.getCoreDescriptor().getCoreContainer().getConfig();
     PluginInfo[] pluginInfos = nodeConfig.getMetricReporterPlugins();
-    metricManager.loadReporters(pluginInfos, core.getResourceLoader(), SolrInfoMBean.Group.core, registryName);
+    metricManager.loadReporters(pluginInfos, core.getResourceLoader(), tag,
+        SolrInfoMBean.Group.core, registryName);
+    if (cloudMode) {
+      metricManager.loadShardReporters(pluginInfos, core);
+    }
   }
 
   /**
@@ -67,12 +96,18 @@ public class SolrCoreMetricManager implements Closeable {
    */
   public void afterCoreSetName() {
     String oldRegistryName = registryName;
-    registryName = createRegistryName(core.getCoreDescriptor().getCollectionName(), core.getName());
+    String oldLeaderRegistryName = leaderRegistryName;
+    initCloudMode();
+    registryName = createRegistryName(cloudMode, collectionName, shardName, replicaName, core.getName());
+    leaderRegistryName = createLeaderRegistryName(cloudMode, collectionName, shardName);
     if (oldRegistryName.equals(registryName)) {
       return;
     }
     // close old reporters
-    metricManager.closeReporters(oldRegistryName);
+    metricManager.closeReporters(oldRegistryName, tag);
+    if (oldLeaderRegistryName != null) {
+      metricManager.closeReporters(oldLeaderRegistryName, tag);
+    }
     // load reporters again, using the new core name
     loadReporters();
   }
@@ -96,7 +131,7 @@ public class SolrCoreMetricManager implements Closeable {
    */
   @Override
   public void close() throws IOException {
-    metricManager.closeReporters(getRegistryName());
+    metricManager.closeReporters(getRegistryName(), tag);
   }
 
   public SolrCore getCore() {
@@ -104,7 +139,7 @@ public class SolrCoreMetricManager implements Closeable {
   }
 
   /**
-   * Retrieves the metric registry name of the manager.
+   * 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
@@ -124,22 +159,74 @@ public class SolrCoreMetricManager implements Closeable {
     return registryName;
   }
 
-  public static String createRegistryName(String collectionName, String coreName) {
-    if (collectionName == null || (collectionName != null && !coreName.startsWith(collectionName + "_"))) {
-      // single core, or unknown naming scheme
+  /**
+   * Metric registry name for leader metrics. This is null if not in cloud mode.
+   * @return metric registry name for leader metrics
+   */
+  public String getLeaderRegistryName() {
+    return leaderRegistryName;
+  }
+
+  /**
+   * Return a tag specific to this instance.
+   */
+  public String getTag() {
+    return tag;
+  }
+
+  public static String createRegistryName(boolean cloud, String collectionName, String shardName, String replicaName, String coreName) {
+    if (cloud) { // build registry name from logical names
+      return SolrMetricManager.getRegistryName(SolrInfoMBean.Group.core, collectionName, shardName, replicaName);
+    } else {
       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;
+  }
+
+  /**
+   * This method is used by {@link org.apache.solr.core.CoreContainer#rename(String, String)}.
+   * @param aCore existing core with old name
+   * @param coreName new name
+   * @return new registry name
+   */
+  public static String createRegistryName(SolrCore aCore, String coreName) {
+    CloudDescriptor cd = aCore.getCoreDescriptor().getCloudDescriptor();
+    String replicaName = null;
+    if (cd != null) {
+      replicaName = parseReplicaName(cd.getCollectionName(), coreName);
+    }
+    return createRegistryName(
+        cd != null,
+        cd != null ? cd.getCollectionName() : null,
+        cd != null ? cd.getShardId() : null,
+        replicaName,
+        coreName
+        );
+  }
+
+  public static String parseReplicaName(String collectionName, String coreName) {
+    if (collectionName == null || !coreName.startsWith(collectionName)) {
+      return null;
+    } else {
+      // split "collection1_shard1_1_replica1" into parts
+      if (coreName.length() > collectionName.length()) {
+        String str = coreName.substring(collectionName.length() + 1);
+        int pos = str.lastIndexOf("_replica");
+        if (pos == -1) { // ?? no _replicaN part ??
+          return str;
+        } else {
+          return str.substring(pos + 1);
+        }
+      } else {
+        return null;
+      }
+    }
+  }
+
+  public static String createLeaderRegistryName(boolean cloud, String collectionName, String shardName) {
+    if (cloud) {
+      return SolrMetricManager.getRegistryName(SolrInfoMBean.Group.collection, collectionName, shardName, "leader");
     } else {
-      shard = str.substring(0, pos);
-      replica = str.substring(pos + 1);
+      return null;
     }
-    return SolrMetricManager.getRegistryName(SolrInfoMBean.Group.core, collectionName, shard, replica);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d7bc947/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 cac5389..3a4c3fe 100644
--- a/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
@@ -18,9 +18,13 @@ package org.apache.solr.metrics;
 
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
@@ -29,6 +33,9 @@ import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Pattern;
+import java.util.regex.PatternSyntaxException;
+import java.util.stream.Collectors;
 
 import com.codahale.metrics.Counter;
 import com.codahale.metrics.Histogram;
@@ -39,9 +46,14 @@ import com.codahale.metrics.MetricRegistry;
 import com.codahale.metrics.MetricSet;
 import com.codahale.metrics.SharedMetricRegistries;
 import com.codahale.metrics.Timer;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.CoreContainer;
 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.apache.solr.metrics.reporters.solr.SolrClusterReporter;
+import org.apache.solr.metrics.reporters.solr.SolrShardReporter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -87,27 +99,39 @@ public class SolrMetricManager {
   private final Lock reportersLock = new ReentrantLock();
   private final Lock swapLock = new ReentrantLock();
 
+  public static final int DEFAULT_CLOUD_REPORTER_PERIOD = 60;
+
   public SolrMetricManager() { }
 
   /**
    * An implementation of {@link MetricFilter} that selects metrics
-   * with names that start with a prefix.
+   * with names that start with one of prefixes.
    */
   public static class PrefixFilter implements MetricFilter {
-    private final String[] prefixes;
+    private final Set<String> prefixes = new HashSet<>();
     private final Set<String> matched = new HashSet<>();
     private boolean allMatch = false;
 
     /**
-     * Create a filter that uses the provided prefix.
+     * Create a filter that uses the provided prefixes.
      * @param prefixes prefixes to use, must not be null. If empty then any
      *               name will match, if not empty then match on any prefix will
      *                 succeed (logical OR).
      */
     public PrefixFilter(String... prefixes) {
       Objects.requireNonNull(prefixes);
-      this.prefixes = prefixes;
-      if (prefixes.length == 0) {
+      if (prefixes.length > 0) {
+        this.prefixes.addAll(Arrays.asList(prefixes));
+      }
+      if (this.prefixes.isEmpty()) {
+        allMatch = true;
+      }
+    }
+
+    public PrefixFilter(Collection<String> prefixes) {
+      Objects.requireNonNull(prefixes);
+      this.prefixes.addAll(prefixes);
+      if (this.prefixes.isEmpty()) {
         allMatch = true;
       }
     }
@@ -141,6 +165,85 @@ public class SolrMetricManager {
     public void reset() {
       matched.clear();
     }
+
+    @Override
+    public String toString() {
+      return "PrefixFilter{" +
+          "prefixes=" + prefixes +
+          '}';
+    }
+  }
+
+  /**
+   * An implementation of {@link MetricFilter} that selects metrics
+   * with names that match regular expression patterns.
+   */
+  public static class RegexFilter implements MetricFilter {
+    private final Set<Pattern> compiledPatterns = new HashSet<>();
+    private final Set<String> matched = new HashSet<>();
+    private boolean allMatch = false;
+
+    /**
+     * Create a filter that uses the provided prefix.
+     * @param patterns regex patterns to use, must not be null. If empty then any
+     *               name will match, if not empty then match on any pattern will
+     *                 succeed (logical OR).
+     */
+    public RegexFilter(String... patterns) throws PatternSyntaxException {
+      this(patterns != null ? Arrays.asList(patterns) : Collections.emptyList());
+    }
+
+    public RegexFilter(Collection<String> patterns) throws PatternSyntaxException {
+      Objects.requireNonNull(patterns);
+      if (patterns.isEmpty()) {
+        allMatch = true;
+        return;
+      }
+      patterns.forEach(p -> {
+        Pattern pattern = Pattern.compile(p);
+        compiledPatterns.add(pattern);
+      });
+      if (patterns.isEmpty()) {
+        allMatch = true;
+      }
+    }
+
+    @Override
+    public boolean matches(String name, Metric metric) {
+      if (allMatch) {
+        matched.add(name);
+        return true;
+      }
+      for (Pattern p : compiledPatterns) {
+        if (p.matcher(name).matches()) {
+          matched.add(name);
+          return true;
+        }
+      }
+      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();
+    }
+
+    @Override
+    public String toString() {
+      return "RegexFilter{" +
+          "compiledPatterns=" + compiledPatterns +
+          '}';
+    }
   }
 
   /**
@@ -150,7 +253,40 @@ public class SolrMetricManager {
     Set<String> set = new HashSet<>();
     set.addAll(registries.keySet());
     set.addAll(SharedMetricRegistries.names());
-    return Collections.unmodifiableSet(set);
+    return set;
+  }
+
+  /**
+   * Return set of existing registry names that match a regex pattern
+   * @param patterns regex patterns. NOTE: users need to make sure that patterns that
+   *                 don't start with a wildcard use the full registry name starting with
+   *                 {@link #REGISTRY_NAME_PREFIX}
+   * @return set of existing registry names where at least one pattern matched.
+   */
+  public Set<String> registryNames(String... patterns) throws PatternSyntaxException {
+    if (patterns == null || patterns.length == 0) {
+      return registryNames();
+    }
+    List<Pattern> compiled = new ArrayList<>();
+    for (String pattern : patterns) {
+      compiled.add(Pattern.compile(pattern));
+    }
+    return registryNames((Pattern[])compiled.toArray(new Pattern[compiled.size()]));
+  }
+
+  public Set<String> registryNames(Pattern... patterns) {
+    Set<String> allNames = registryNames();
+    if (patterns == null || patterns.length == 0) {
+      return allNames;
+    }
+    return allNames.stream().filter(s -> {
+      for (Pattern p : patterns) {
+        if (p.matcher(s).matches()) {
+          return true;
+        }
+      }
+      return false;
+    }).collect(Collectors.toSet());
   }
 
   /**
@@ -209,7 +345,7 @@ public class SolrMetricManager {
    */
   public void removeRegistry(String registry) {
     // close any reporters for this registry first
-    closeReporters(registry);
+    closeReporters(registry, null);
     // make sure we use a name with prefix, with overrides
     registry = overridableRegistryName(registry);
     if (isSharedRegistry(registry)) {
@@ -490,10 +626,12 @@ public class SolrMetricManager {
    * the list. If both attributes are present then only "group" attribute will be processed.
    * @param pluginInfos plugin configurations
    * @param loader resource loader
+   * @param tag optional tag for the reporters, to distinguish reporters logically created for different parent
+   *            component instances.
    * @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) {
+  public void loadReporters(PluginInfo[] pluginInfos, SolrResourceLoader loader, String tag, SolrInfoMBean.Group group, String... registryNames) {
     if (pluginInfos == null || pluginInfos.length == 0) {
       return;
     }
@@ -533,7 +671,7 @@ public class SolrMetricManager {
         }
       }
       try {
-        loadReporter(registryName, loader, info);
+        loadReporter(registryName, loader, info, tag);
       } catch (Exception e) {
         log.warn("Error loading metrics reporter, plugin info: " + info, e);
       }
@@ -545,9 +683,12 @@ public class SolrMetricManager {
    * @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.
+   * @param tag optional tag for the reporter, to distinguish reporters logically created for different parent
+   *            component instances.
+   * @return instance of newly created and registered reporter
    * @throws Exception if any argument is missing or invalid
    */
-  public void loadReporter(String registry, SolrResourceLoader loader, PluginInfo pluginInfo) throws Exception {
+  public SolrMetricReporter loadReporter(String registry, SolrResourceLoader loader, PluginInfo pluginInfo, String tag) 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);
@@ -558,14 +699,19 @@ public class SolrMetricManager {
         pluginInfo.className,
         SolrMetricReporter.class,
         new String[0],
-        new Class[] { SolrMetricManager.class, String.class },
-        new Object[] { this, registry }
+        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);
     }
+    registerReporter(registry, pluginInfo.name, tag, reporter);
+    return reporter;
+  }
+
+  private void registerReporter(String registry, String name, String tag, SolrMetricReporter reporter) throws Exception {
     try {
       if (!reportersLock.tryLock(10, TimeUnit.SECONDS)) {
         throw new Exception("Could not obtain lock to modify reporters registry: " + registry);
@@ -579,12 +725,15 @@ public class SolrMetricManager {
         perRegistry = new HashMap<>();
         reporters.put(registry, perRegistry);
       }
-      SolrMetricReporter oldReporter = perRegistry.get(pluginInfo.name);
+      if (tag != null && !tag.isEmpty()) {
+        name = name + "@" + tag;
+      }
+      SolrMetricReporter oldReporter = perRegistry.get(name);
       if (oldReporter != null) { // close it
-        log.info("Replacing existing reporter '" + pluginInfo.name + "' in registry '" + registry + "': " + oldReporter.toString());
+        log.info("Replacing existing reporter '" + name + "' in registry '" + registry + "': " + oldReporter.toString());
         oldReporter.close();
       }
-      perRegistry.put(pluginInfo.name, reporter);
+      perRegistry.put(name, reporter);
 
     } finally {
       reportersLock.unlock();
@@ -595,9 +744,11 @@ public class SolrMetricManager {
    * Close and unregister a named {@link SolrMetricReporter} for a registry.
    * @param registry registry name
    * @param name reporter name
+   * @param tag optional tag for the reporter, to distinguish reporters logically created for different parent
+   *            component instances.
    * @return true if a named reporter existed and was closed.
    */
-  public boolean closeReporter(String registry, String name) {
+  public boolean closeReporter(String registry, String name, String tag) {
     // make sure we use a name with prefix, with overrides
     registry = overridableRegistryName(registry);
     try {
@@ -614,6 +765,9 @@ public class SolrMetricManager {
       if (perRegistry == null) {
         return false;
       }
+      if (tag != null && !tag.isEmpty()) {
+        name = name + "@" + tag;
+      }
       SolrMetricReporter reporter = perRegistry.remove(name);
       if (reporter == null) {
         return false;
@@ -635,6 +789,17 @@ public class SolrMetricManager {
    * @return names of closed reporters
    */
   public Set<String> closeReporters(String registry) {
+    return closeReporters(registry, null);
+  }
+
+  /**
+   * Close and unregister all {@link SolrMetricReporter}-s for a registry.
+   * @param registry registry name
+   * @param tag optional tag for the reporter, to distinguish reporters logically created for different parent
+   *            component instances.
+   * @return names of closed reporters
+   */
+  public Set<String> closeReporters(String registry, String tag) {
     // make sure we use a name with prefix, with overrides
     registry = overridableRegistryName(registry);
     try {
@@ -646,18 +811,28 @@ public class SolrMetricManager {
       log.warn("Interrupted while trying to obtain lock to modify reporters registry: " + registry);
       return Collections.emptySet();
     }
-    log.info("Closing metric reporters for: " + registry);
+    log.info("Closing metric reporters for registry=" + registry + ", tag=" + tag);
     try {
-      Map<String, SolrMetricReporter> perRegistry = reporters.remove(registry);
+      Map<String, SolrMetricReporter> perRegistry = reporters.get(registry);
       if (perRegistry != null) {
-        for (SolrMetricReporter reporter : perRegistry.values()) {
+        Set<String> names = new HashSet<>(perRegistry.keySet());
+        Set<String> removed = new HashSet<>();
+        names.forEach(name -> {
+          if (tag != null && !tag.isEmpty() && !name.endsWith("@" + tag)) {
+            return;
+          }
+          SolrMetricReporter reporter = perRegistry.remove(name);
           try {
             reporter.close();
           } catch (IOException ioe) {
             log.warn("Exception closing reporter " + reporter, ioe);
           }
+          removed.add(name);
+        });
+        if (removed.size() == names.size()) {
+          reporters.remove(registry);
         }
-        return perRegistry.keySet();
+        return removed;
       } else {
         return Collections.emptySet();
       }
@@ -695,4 +870,114 @@ public class SolrMetricManager {
       reportersLock.unlock();
     }
   }
+
+  private List<PluginInfo> prepareCloudPlugins(PluginInfo[] pluginInfos, String group, String className,
+                                                      Map<String, String> defaultAttributes,
+                                                      Map<String, Object> defaultInitArgs,
+                                                      PluginInfo defaultPlugin) {
+    List<PluginInfo> result = new ArrayList<>();
+    if (pluginInfos == null) {
+      pluginInfos = new PluginInfo[0];
+    }
+    for (PluginInfo info : pluginInfos) {
+      String groupAttr = info.attributes.get("group");
+      if (!group.equals(groupAttr)) {
+        continue;
+      }
+      info = preparePlugin(info, className, defaultAttributes, defaultInitArgs);
+      if (info != null) {
+        result.add(info);
+      }
+    }
+    if (result.isEmpty() && defaultPlugin != null) {
+      defaultPlugin = preparePlugin(defaultPlugin, className, defaultAttributes, defaultInitArgs);
+      if (defaultPlugin != null) {
+        result.add(defaultPlugin);
+      }
+    }
+    return result;
+  }
+
+  private PluginInfo preparePlugin(PluginInfo info, String className, Map<String, String> defaultAttributes,
+                                   Map<String, Object> defaultInitArgs) {
+    if (info == null) {
+      return null;
+    }
+    String classNameAttr = info.attributes.get("class");
+    if (className != null) {
+      if (classNameAttr != null && !className.equals(classNameAttr)) {
+        log.warn("Conflicting class name attributes, expected " + className + " but was " + classNameAttr + ", skipping " + info);
+        return null;
+      }
+    }
+
+    Map<String, String> attrs = new HashMap<>(info.attributes);
+    defaultAttributes.forEach((k, v) -> {
+      if (!attrs.containsKey(k)) {
+        attrs.put(k, v);
+      }
+    });
+    attrs.put("class", className);
+    Map<String, Object> initArgs = new HashMap<>();
+    if (info.initArgs != null) {
+      initArgs.putAll(info.initArgs.asMap(10));
+    }
+    defaultInitArgs.forEach((k, v) -> {
+      if (!initArgs.containsKey(k)) {
+        initArgs.put(k, v);
+      }
+    });
+    return new PluginInfo(info.type, attrs, new NamedList(initArgs), null);
+  }
+
+  public void loadShardReporters(PluginInfo[] pluginInfos, SolrCore core) {
+    // don't load for non-cloud cores
+    if (core.getCoreDescriptor().getCloudDescriptor() == null) {
+      return;
+    }
+    // prepare default plugin if none present in the config
+    Map<String, String> attrs = new HashMap<>();
+    attrs.put("name", "shardDefault");
+    attrs.put("group", SolrInfoMBean.Group.shard.toString());
+    Map<String, Object> initArgs = new HashMap<>();
+    initArgs.put("period", DEFAULT_CLOUD_REPORTER_PERIOD);
+
+    String registryName = core.getCoreMetricManager().getRegistryName();
+    // collect infos and normalize
+    List<PluginInfo> infos = prepareCloudPlugins(pluginInfos, SolrInfoMBean.Group.shard.toString(), SolrShardReporter.class.getName(),
+        attrs, initArgs, null);
+    for (PluginInfo info : infos) {
+      try {
+        SolrMetricReporter reporter = loadReporter(registryName, core.getResourceLoader(), info,
+            String.valueOf(core.hashCode()));
+        ((SolrShardReporter)reporter).setCore(core);
+      } catch (Exception e) {
+        log.warn("Could not load shard reporter, pluginInfo=" + info, e);
+      }
+    }
+  }
+
+  public void loadClusterReporters(PluginInfo[] pluginInfos, CoreContainer cc) {
+    // don't load for non-cloud instances
+    if (!cc.isZooKeeperAware()) {
+      return;
+    }
+    Map<String, String> attrs = new HashMap<>();
+    attrs.put("name", "clusterDefault");
+    attrs.put("group", SolrInfoMBean.Group.cluster.toString());
+    Map<String, Object> initArgs = new HashMap<>();
+    initArgs.put("period", DEFAULT_CLOUD_REPORTER_PERIOD);
+    List<PluginInfo> infos = prepareCloudPlugins(pluginInfos, SolrInfoMBean.Group.cluster.toString(), SolrClusterReporter.class.getName(),
+        attrs, initArgs, null);
+    String registryName = getRegistryName(SolrInfoMBean.Group.cluster);
+    for (PluginInfo info : infos) {
+      try {
+        SolrMetricReporter reporter = loadReporter(registryName, cc.getResourceLoader(), info, null);
+        ((SolrClusterReporter)reporter).setCoreContainer(cc);
+      } catch (Exception e) {
+        log.warn("Could not load node reporter, pluginInfo=" + info, e);
+      }
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d7bc947/solr/core/src/java/org/apache/solr/metrics/reporters/JmxObjectNameFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/reporters/JmxObjectNameFactory.java b/solr/core/src/java/org/apache/solr/metrics/reporters/JmxObjectNameFactory.java
index 4df5257..1f5b4f0 100644
--- a/solr/core/src/java/org/apache/solr/metrics/reporters/JmxObjectNameFactory.java
+++ b/solr/core/src/java/org/apache/solr/metrics/reporters/JmxObjectNameFactory.java
@@ -41,9 +41,9 @@ public class JmxObjectNameFactory implements ObjectNameFactory {
    * @param additionalProperties additional properties as key, value pairs.
    */
   public JmxObjectNameFactory(String reporterName, String domain, String... additionalProperties) {
-    this.reporterName = reporterName;
+    this.reporterName = reporterName.replaceAll(":", "_");
     this.domain = domain;
-    this.subdomains = domain.split("\\.");
+    this.subdomains = domain.replaceAll(":", "_").split("\\.");
     if (additionalProperties != null && (additionalProperties.length % 2) != 0) {
       throw new IllegalArgumentException("additionalProperties length must be even: " + Arrays.toString(additionalProperties));
     }
@@ -83,7 +83,7 @@ public class JmxObjectNameFactory implements ObjectNameFactory {
         }
         sb.append(','); // separate from other properties
       } else {
-        sb.append(currentDomain);
+        sb.append(currentDomain.replaceAll(":", "_"));
         sb.append(':');
       }
     } else {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d7bc947/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrClusterReporter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrClusterReporter.java b/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrClusterReporter.java
new file mode 100644
index 0000000..846e805
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrClusterReporter.java
@@ -0,0 +1,277 @@
+/*
+ * 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.solr;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.net.MalformedURLException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+import org.apache.http.client.HttpClient;
+import org.apache.solr.cloud.Overseer;
+import org.apache.solr.cloud.ZkController;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.SolrInfoMBean;
+import org.apache.solr.handler.admin.MetricsCollectorHandler;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.metrics.SolrMetricReporter;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This reporter sends selected metrics from local registries to {@link Overseer}.
+ * <p>The following configuration properties are supported:</p>
+ * <ul>
+ *   <li>handler - (optional str) handler path where reports are sent. Default is
+ *   {@link MetricsCollectorHandler#HANDLER_PATH}.</li>
+ *   <li>period - (optional int) how often reports are sent, in seconds. Default is 60. Setting this
+ *   to 0 disables the reporter.</li>
+ *   <li>report - (optional multiple lst) report configuration(s), see below.</li>
+ * </ul>
+ * Each report configuration consist of the following properties:
+ * <ul>
+ *   <li>registry - (required str) regex pattern matching source registries (see {@link SolrMetricManager#registryNames(String...)}),
+ *   may contain capture groups.</li>
+ *   <li>group - (required str) target registry name where metrics will be grouped. This can be a regex pattern that
+ *   contains back-references to capture groups collected by <code>registry</code> pattern</li>
+ *   <li>label - (optional str) optional prefix to prepend to metric names, may contain back-references to
+ *   capture groups collected by <code>registry</code> pattern</li>
+ *   <li>filter - (optional multiple str) regex expression(s) matching selected metrics to be reported.</li>
+ * </ul>
+ * NOTE: this reporter uses predefined "overseer" group, and it's always created even if explicit configuration
+ * is missing. Default configuration uses report specifications from {@link #DEFAULT_REPORTS}.
+ * <p>Example configuration:</p>
+ * <pre>
+ *       &lt;reporter name="test" group="overseer"&gt;
+ *         &lt;str name="handler"&gt;/admin/metrics/collector&lt;/str&gt;
+ *         &lt;int name="period"&gt;11&lt;/int&gt;
+ *         &lt;lst name="report"&gt;
+ *           &lt;str name="group"&gt;overseer&lt;/str&gt;
+ *           &lt;str name="label"&gt;jvm&lt;/str&gt;
+ *           &lt;str name="registry"&gt;solr\.jvm&lt;/str&gt;
+ *           &lt;str name="filter"&gt;memory\.total\..*&lt;/str&gt;
+ *           &lt;str name="filter"&gt;memory\.heap\..*&lt;/str&gt;
+ *           &lt;str name="filter"&gt;os\.SystemLoadAverage&lt;/str&gt;
+ *           &lt;str name="filter"&gt;threads\.count&lt;/str&gt;
+ *         &lt;/lst&gt;
+ *         &lt;lst name="report"&gt;
+ *           &lt;str name="group"&gt;overseer&lt;/str&gt;
+ *           &lt;str name="label"&gt;leader.$1&lt;/str&gt;
+ *           &lt;str name="registry"&gt;solr\.core\.(.*)\.leader&lt;/str&gt;
+ *           &lt;str name="filter"&gt;UPDATE\./update/.*&lt;/str&gt;
+ *         &lt;/lst&gt;
+ *       &lt;/reporter&gt;
+ * </pre>
+ *
+ */
+public class SolrClusterReporter extends SolrMetricReporter {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public static final String CLUSTER_GROUP = SolrMetricManager.overridableRegistryName(SolrInfoMBean.Group.cluster.toString());
+
+  public static final List<SolrReporter.Report> DEFAULT_REPORTS = new ArrayList<SolrReporter.Report>() {{
+    add(new SolrReporter.Report(CLUSTER_GROUP, "jetty",
+        SolrMetricManager.overridableRegistryName(SolrInfoMBean.Group.jetty.toString()),
+        Collections.emptySet())); // all metrics
+    add(new SolrReporter.Report(CLUSTER_GROUP, "jvm",
+        SolrMetricManager.overridableRegistryName(SolrInfoMBean.Group.jvm.toString()),
+        new HashSet<String>() {{
+          add("memory\\.total\\..*");
+          add("memory\\.heap\\..*");
+          add("os\\.SystemLoadAverage");
+          add("os\\.FreePhysicalMemorySize");
+          add("os\\.FreeSwapSpaceSize");
+          add("os\\.OpenFileDescriptorCount");
+          add("threads\\.count");
+        }})); // all metrics
+    // XXX anything interesting here?
+    //add(new SolrReporter.Specification(OVERSEER_GROUP, "node", SolrMetricManager.overridableRegistryName(SolrInfoMBean.Group.node.toString()),
+    //    Collections.emptySet())); // all metrics
+    add(new SolrReporter.Report(CLUSTER_GROUP, "leader.$1", "solr\\.collection\\.(.*)\\.leader",
+        new HashSet<String>(){{
+          add("UPDATE\\./update/.*");
+          add("QUERY\\./select.*");
+          add("INDEX\\..*");
+          add("TLOG\\..*");
+    }}));
+  }};
+
+  private String handler = MetricsCollectorHandler.HANDLER_PATH;
+  private int period = SolrMetricManager.DEFAULT_CLOUD_REPORTER_PERIOD;
+  private List<SolrReporter.Report> reports = new ArrayList<>();
+
+  private SolrReporter reporter;
+
+  /**
+   * Create a reporter for metrics managed in a named registry.
+   *
+   * @param metricManager metric manager
+   * @param registryName  this is ignored
+   */
+  public SolrClusterReporter(SolrMetricManager metricManager, String registryName) {
+    super(metricManager, registryName);
+  }
+
+  public void setHandler(String handler) {
+    this.handler = handler;
+  }
+
+  public void setPeriod(int period) {
+    this.period = period;
+  }
+
+  public void setReport(List<Map> reportConfig) {
+    if (reportConfig == null || reportConfig.isEmpty()) {
+      return;
+    }
+    reportConfig.forEach(map -> {
+      SolrReporter.Report r = SolrReporter.Report.fromMap(map);
+      if (r != null) {
+        reports.add(r);
+      }
+    });
+  }
+
+  // for unit tests
+  int getPeriod() {
+    return period;
+  }
+
+  List<SolrReporter.Report> getReports() {
+    return reports;
+  }
+
+  @Override
+  protected void validate() throws IllegalStateException {
+    if (period < 1) {
+      log.info("Turning off node reporter, period=" + period);
+    }
+    if (reports.isEmpty()) { // set defaults
+      reports = DEFAULT_REPORTS;
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (reporter != null) {
+      reporter.close();;
+    }
+  }
+
+  public void setCoreContainer(CoreContainer cc) {
+    if (reporter != null) {
+      reporter.close();;
+    }
+    // start reporter only in cloud mode
+    if (!cc.isZooKeeperAware()) {
+      log.warn("Not ZK-aware, not starting...");
+      return;
+    }
+    if (period < 1) { // don't start it
+      return;
+    }
+    HttpClient httpClient = cc.getUpdateShardHandler().getHttpClient();
+    ZkController zk = cc.getZkController();
+    String reporterId = zk.getNodeName();
+    reporter = SolrReporter.Builder.forReports(metricManager, reports)
+        .convertRatesTo(TimeUnit.SECONDS)
+        .convertDurationsTo(TimeUnit.MILLISECONDS)
+        .withHandler(handler)
+        .withReporterId(reporterId)
+        .cloudClient(false) // we want to send reports specifically to a selected leader instance
+        .skipAggregateValues(true) // we don't want to transport details of aggregates
+        .skipHistograms(true) // we don't want to transport histograms
+        .build(httpClient, new OverseerUrlSupplier(zk));
+
+    reporter.start(period, TimeUnit.SECONDS);
+  }
+
+  // TODO: fix this when there is an elegant way to retrieve URL of a node that runs Overseer leader.
+  // package visibility for unit tests
+  static class OverseerUrlSupplier implements Supplier<String> {
+    private static final long DEFAULT_INTERVAL = 30000000; // 30s
+    private ZkController zk;
+    private String lastKnownUrl = null;
+    private long lastCheckTime = 0;
+    private long interval = DEFAULT_INTERVAL;
+
+    OverseerUrlSupplier(ZkController zk) {
+      this.zk = zk;
+    }
+
+    @Override
+    public String get() {
+      if (zk == null) {
+        return null;
+      }
+      // primitive caching for lastKnownUrl
+      long now = System.nanoTime();
+      if (lastKnownUrl != null && (now - lastCheckTime) < interval) {
+        return lastKnownUrl;
+      }
+      if (!zk.isConnected()) {
+        return lastKnownUrl;
+      }
+      lastCheckTime = now;
+      SolrZkClient zkClient = zk.getZkClient();
+      ZkNodeProps props;
+      try {
+        props = ZkNodeProps.load(zkClient.getData(
+            Overseer.OVERSEER_ELECT + "/leader", null, null, true));
+      } catch (KeeperException e) {
+        log.warn("Could not obtain overseer's address, skipping.", e);
+        return lastKnownUrl;
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        return lastKnownUrl;
+      }
+      if (props == null) {
+        return lastKnownUrl;
+      }
+      String oid = props.getStr("id");
+      if (oid == null) {
+        return lastKnownUrl;
+      }
+      String[] ids = oid.split("-");
+      if (ids.length != 3) { // unknown format
+        log.warn("Unknown format of leader id, skipping: " + oid);
+        return lastKnownUrl;
+      }
+      // convert nodeName back to URL
+      String url = zk.getZkStateReader().getBaseUrlForNodeName(ids[1]);
+      // check that it's parseable
+      try {
+        new java.net.URL(url);
+      } catch (MalformedURLException mue) {
+        log.warn("Malformed Overseer's leader URL: url", mue);
+        return lastKnownUrl;
+      }
+      lastKnownUrl = url;
+      return url;
+    }
+  }
+
+}


[20/43] lucene-solr:feature/autoscaling: SOLR-10254: Fix pre-commit

Posted by sh...@apache.org.
SOLR-10254: Fix pre-commit


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

Branch: refs/heads/feature/autoscaling
Commit: c85aac2a65472d0d80050a703c99844e694c1584
Parents: 682c6a7
Author: Joel Bernstein <jb...@apache.org>
Authored: Wed Mar 8 21:35:24 2017 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Wed Mar 8 21:35:24 2017 -0500

----------------------------------------------------------------------
 .../solr/client/solrj/io/stream/SignificantTermsStream.java | 9 ---------
 1 file changed, 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c85aac2a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SignificantTermsStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SignificantTermsStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SignificantTermsStream.java
index 2acee51..101a71d 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SignificantTermsStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SignificantTermsStream.java
@@ -19,7 +19,6 @@ package org.apache.solr.client.solrj.io.stream;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
@@ -27,13 +26,10 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
-import java.util.Random;
-import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
 
-import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.io.SolrClientCache;
 import org.apache.solr.client.solrj.io.Tuple;
@@ -48,11 +44,6 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionValue;
 import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
 import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.client.solrj.response.QueryResponse;
-import org.apache.solr.common.cloud.ClusterState;
-import org.apache.solr.common.cloud.Replica;
-import org.apache.solr.common.cloud.Slice;
-import org.apache.solr.common.cloud.ZkCoreNodeProps;
-import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.NamedList;


[33/43] lucene-solr:feature/autoscaling: LUCENE-7740: Refactor Range Fields to remove Field suffix (e.g., DoubleRange), move InetAddressRange and InetAddressPoint from sandbox to misc module, and refactor all other range fields from sandbox to core.

Posted by sh...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/core/src/test/org/apache/lucene/search/TestIntRangeFieldQueries.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestIntRangeFieldQueries.java b/lucene/core/src/test/org/apache/lucene/search/TestIntRangeFieldQueries.java
new file mode 100644
index 0000000..14771c9
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/TestIntRangeFieldQueries.java
@@ -0,0 +1,251 @@
+/*
+ * 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 org.apache.lucene.document.Document;
+import org.apache.lucene.document.IntRange;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.store.Directory;
+
+/**
+ * Random testing for IntRange Queries.
+ */
+public class TestIntRangeFieldQueries extends BaseRangeFieldQueryTestCase {
+  private static final String FIELD_NAME = "intRangeField";
+
+  private int nextIntInternal() {
+    if (rarely()) {
+      return random().nextBoolean() ? Integer.MAX_VALUE : Integer.MIN_VALUE;
+    }
+    int max = Integer.MAX_VALUE / 2;
+    return (max + max) * random().nextInt() - max;
+  }
+
+  @Override
+  protected Range nextRange(int dimensions) throws Exception {
+    int[] min = new int[dimensions];
+    int[] max = new int[dimensions];
+
+    int minV, maxV;
+    for (int d=0; d<dimensions; ++d) {
+      minV = nextIntInternal();
+      maxV = nextIntInternal();
+      min[d] = Math.min(minV, maxV);
+      max[d] = Math.max(minV, maxV);
+    }
+    return new IntTestRange(min, max);
+  }
+
+  @Override
+  protected org.apache.lucene.document.IntRange newRangeField(Range r) {
+    return new IntRange(FIELD_NAME, ((IntTestRange)r).min, ((IntTestRange)r).max);
+  }
+
+  @Override
+  protected Query newIntersectsQuery(Range r) {
+    return IntRange.newIntersectsQuery(FIELD_NAME, ((IntTestRange)r).min, ((IntTestRange)r).max);
+  }
+
+  @Override
+  protected Query newContainsQuery(Range r) {
+    return IntRange.newContainsQuery(FIELD_NAME, ((IntTestRange)r).min, ((IntTestRange)r).max);
+  }
+
+  @Override
+  protected Query newWithinQuery(Range r) {
+    return IntRange.newWithinQuery(FIELD_NAME, ((IntTestRange)r).min, ((IntTestRange)r).max);
+  }
+
+  @Override
+  protected Query newCrossesQuery(Range r) {
+    return IntRange.newCrossesQuery(FIELD_NAME, ((IntTestRange)r).min, ((IntTestRange)r).max);
+  }
+
+  /** Basic test */
+  public void testBasics() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+
+    // intersects (within)
+    Document document = new Document();
+    document.add(new IntRange(FIELD_NAME, new int[] {-10, -10}, new int[] {9, 10}));
+    writer.addDocument(document);
+
+    // intersects (crosses)
+    document = new Document();
+    document.add(new IntRange(FIELD_NAME, new int[] {10, -10}, new int[] {20, 10}));
+    writer.addDocument(document);
+
+    // intersects (contains / crosses)
+    document = new Document();
+    document.add(new IntRange(FIELD_NAME, new int[] {-20, -20}, new int[] {30, 30}));
+    writer.addDocument(document);
+
+    // intersects (within)
+    document = new Document();
+    document.add(new IntRange(FIELD_NAME, new int[] {-11, -11}, new int[] {1, 11}));
+    writer.addDocument(document);
+
+    // intersects (crosses)
+    document = new Document();
+    document.add(new IntRange(FIELD_NAME, new int[] {12, 1}, new int[] {15, 29}));
+    writer.addDocument(document);
+
+    // disjoint
+    document = new Document();
+    document.add(new IntRange(FIELD_NAME, new int[] {-122, 1}, new int[] {-115, 29}));
+    writer.addDocument(document);
+
+    // intersects (crosses)
+    document = new Document();
+    document.add(new IntRange(FIELD_NAME, new int[] {Integer.MIN_VALUE, 1}, new int[] {-11, 29}));
+    writer.addDocument(document);
+
+    // equal (within, contains, intersects)
+    document = new Document();
+    document.add(new IntRange(FIELD_NAME, new int[] {-11, -15}, new int[] {15, 20}));
+    writer.addDocument(document);
+
+    // search
+    IndexReader reader = writer.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+    assertEquals(7, searcher.count(IntRange.newIntersectsQuery(FIELD_NAME,
+        new int[] {-11, -15}, new int[] {15, 20})));
+    assertEquals(3, searcher.count(IntRange.newWithinQuery(FIELD_NAME,
+        new int[] {-11, -15}, new int[] {15, 20})));
+    assertEquals(2, searcher.count(IntRange.newContainsQuery(FIELD_NAME,
+        new int[] {-11, -15}, new int[] {15, 20})));
+    assertEquals(4, searcher.count(IntRange.newCrossesQuery(FIELD_NAME,
+        new int[] {-11, -15}, new int[] {15, 20})));
+
+    reader.close();
+    writer.close();
+    dir.close();
+  }
+
+  /** IntRange test class implementation - use to validate IntRange */
+  private class IntTestRange extends Range {
+    int[] min;
+    int[] max;
+
+    IntTestRange(int[] min, int[] max) {
+      assert min != null && max != null && min.length > 0 && max.length > 0
+          : "test box: min/max cannot be null or empty";
+      assert min.length == max.length : "test box: min/max length do not agree";
+      this.min = min;
+      this.max = max;
+    }
+
+    @Override
+    protected int numDimensions() {
+      return min.length;
+    }
+
+    @Override
+    protected Integer getMin(int dim) {
+      return min[dim];
+    }
+
+    @Override
+    protected void setMin(int dim, Object val) {
+      int v = (Integer)val;
+      if (min[dim] < v) {
+        max[dim] = v;
+      } else {
+        min[dim] = v;
+      }
+    }
+
+    @Override
+    protected Integer getMax(int dim) {
+      return max[dim];
+    }
+
+    @Override
+    protected void setMax(int dim, Object val) {
+      int v = (Integer)val;
+      if (max[dim] > v) {
+        min[dim] = v;
+      } else {
+        max[dim] = v;
+      }
+    }
+
+    @Override
+    protected boolean isEqual(Range other) {
+      IntTestRange o = (IntTestRange)other;
+      return Arrays.equals(min, o.min) && Arrays.equals(max, o.max);
+    }
+
+    @Override
+    protected boolean isDisjoint(Range o) {
+      IntTestRange other = (IntTestRange)o;
+      for (int d=0; d<this.min.length; ++d) {
+        if (this.min[d] > other.max[d] || this.max[d] < other.min[d]) {
+          // disjoint:
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    protected boolean isWithin(Range o) {
+      IntTestRange other = (IntTestRange)o;
+      for (int d=0; d<this.min.length; ++d) {
+        if ((this.min[d] >= other.min[d] && this.max[d] <= other.max[d]) == false) {
+          // not within:
+          return false;
+        }
+      }
+      return true;
+    }
+
+    @Override
+    protected boolean contains(Range o) {
+      IntTestRange other = (IntTestRange) o;
+      for (int d=0; d<this.min.length; ++d) {
+        if ((this.min[d] <= other.min[d] && this.max[d] >= other.max[d]) == false) {
+          // not contains:
+          return false;
+        }
+      }
+      return true;
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder b = new StringBuilder();
+      b.append("Box(");
+      b.append(min[0]);
+      b.append(" TO ");
+      b.append(max[0]);
+      for (int d=1; d<min.length; ++d) {
+        b.append(", ");
+        b.append(min[d]);
+        b.append(" TO ");
+        b.append(max[d]);
+      }
+      b.append(")");
+
+      return b.toString();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/core/src/test/org/apache/lucene/search/TestLongRangeFieldQueries.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestLongRangeFieldQueries.java b/lucene/core/src/test/org/apache/lucene/search/TestLongRangeFieldQueries.java
new file mode 100644
index 0000000..60d7ea3
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/TestLongRangeFieldQueries.java
@@ -0,0 +1,251 @@
+/*
+ * 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 org.apache.lucene.document.Document;
+import org.apache.lucene.document.LongRange;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.store.Directory;
+
+/**
+ * Random testing for LongRange Queries.
+ */
+public class TestLongRangeFieldQueries extends BaseRangeFieldQueryTestCase {
+  private static final String FIELD_NAME = "longRangeField";
+
+  private long nextLongInternal() {
+    if (rarely()) {
+      return random().nextBoolean() ? Long.MAX_VALUE : Long.MIN_VALUE;
+    }
+    long max = Long.MAX_VALUE / 2;
+    return (max + max) * random().nextLong() - max;
+  }
+
+  @Override
+  protected Range nextRange(int dimensions) throws Exception {
+    long[] min = new long[dimensions];
+    long[] max = new long[dimensions];
+
+    long minV, maxV;
+    for (int d=0; d<dimensions; ++d) {
+      minV = nextLongInternal();
+      maxV = nextLongInternal();
+      min[d] = Math.min(minV, maxV);
+      max[d] = Math.max(minV, maxV);
+    }
+    return new LongTestRange(min, max);
+  }
+
+  @Override
+  protected LongRange newRangeField(Range r) {
+    return new LongRange(FIELD_NAME, ((LongTestRange)r).min, ((LongTestRange)r).max);
+  }
+
+  @Override
+  protected Query newIntersectsQuery(Range r) {
+    return LongRange.newIntersectsQuery(FIELD_NAME, ((LongTestRange)r).min, ((LongTestRange)r).max);
+  }
+
+  @Override
+  protected Query newContainsQuery(Range r) {
+    return LongRange.newContainsQuery(FIELD_NAME, ((LongTestRange)r).min, ((LongTestRange)r).max);
+  }
+
+  @Override
+  protected Query newWithinQuery(Range r) {
+    return LongRange.newWithinQuery(FIELD_NAME, ((LongTestRange)r).min, ((LongTestRange)r).max);
+  }
+
+  @Override
+  protected Query newCrossesQuery(Range r) {
+    return LongRange.newCrossesQuery(FIELD_NAME, ((LongTestRange)r).min, ((LongTestRange)r).max);
+  }
+
+  /** Basic test */
+  public void testBasics() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+
+    // intersects (within)
+    Document document = new Document();
+    document.add(new LongRange(FIELD_NAME, new long[] {-10, -10}, new long[] {9, 10}));
+    writer.addDocument(document);
+
+    // intersects (crosses)
+    document = new Document();
+    document.add(new LongRange(FIELD_NAME, new long[] {10, -10}, new long[] {20, 10}));
+    writer.addDocument(document);
+
+    // intersects (contains, crosses)
+    document = new Document();
+    document.add(new LongRange(FIELD_NAME, new long[] {-20, -20}, new long[] {30, 30}));
+    writer.addDocument(document);
+
+    // intersects (within)
+    document = new Document();
+    document.add(new LongRange(FIELD_NAME, new long[] {-11, -11}, new long[] {1, 11}));
+    writer.addDocument(document);
+
+    // intersects (crosses)
+    document = new Document();
+    document.add(new LongRange(FIELD_NAME, new long[] {12, 1}, new long[] {15, 29}));
+    writer.addDocument(document);
+
+    // disjoint
+    document = new Document();
+    document.add(new LongRange(FIELD_NAME, new long[] {-122, 1}, new long[] {-115, 29}));
+    writer.addDocument(document);
+
+    // intersects (crosses)
+    document = new Document();
+    document.add(new LongRange(FIELD_NAME, new long[] {Long.MIN_VALUE, 1}, new long[] {-11, 29}));
+    writer.addDocument(document);
+
+    // equal (within, contains, intersects)
+    document = new Document();
+    document.add(new LongRange(FIELD_NAME, new long[] {-11, -15}, new long[] {15, 20}));
+    writer.addDocument(document);
+
+    // search
+    IndexReader reader = writer.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+    assertEquals(7, searcher.count(LongRange.newIntersectsQuery(FIELD_NAME,
+        new long[] {-11, -15}, new long[] {15, 20})));
+    assertEquals(3, searcher.count(LongRange.newWithinQuery(FIELD_NAME,
+        new long[] {-11, -15}, new long[] {15, 20})));
+    assertEquals(2, searcher.count(LongRange.newContainsQuery(FIELD_NAME,
+        new long[] {-11, -15}, new long[] {15, 20})));
+    assertEquals(4, searcher.count(LongRange.newCrossesQuery(FIELD_NAME,
+        new long[] {-11, -15}, new long[] {15, 20})));
+
+    reader.close();
+    writer.close();
+    dir.close();
+  }
+
+  /** LongRange test class implementation - use to validate LongRange */
+  private class LongTestRange extends Range {
+    long[] min;
+    long[] max;
+
+    LongTestRange(long[] min, long[] max) {
+      assert min != null && max != null && min.length > 0 && max.length > 0
+          : "test box: min/max cannot be null or empty";
+      assert min.length == max.length : "test box: min/max length do not agree";
+      this.min = min;
+      this.max = max;
+    }
+
+    @Override
+    protected int numDimensions() {
+      return min.length;
+    }
+
+    @Override
+    protected Long getMin(int dim) {
+      return min[dim];
+    }
+
+    @Override
+    protected void setMin(int dim, Object val) {
+      long v = (Long)val;
+      if (min[dim] < v) {
+        max[dim] = v;
+      } else {
+        min[dim] = v;
+      }
+    }
+
+    @Override
+    protected Long getMax(int dim) {
+      return max[dim];
+    }
+
+    @Override
+    protected void setMax(int dim, Object val) {
+      long v = (Long)val;
+      if (max[dim] > v) {
+        min[dim] = v;
+      } else {
+        max[dim] = v;
+      }
+    }
+
+    @Override
+    protected boolean isEqual(Range other) {
+      LongTestRange o = (LongTestRange)other;
+      return Arrays.equals(min, o.min) && Arrays.equals(max, o.max);
+    }
+
+    @Override
+    protected boolean isDisjoint(Range o) {
+      LongTestRange other = (LongTestRange)o;
+      for (int d=0; d<this.min.length; ++d) {
+        if (this.min[d] > other.max[d] || this.max[d] < other.min[d]) {
+          // disjoint:
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    protected boolean isWithin(Range o) {
+      LongTestRange other = (LongTestRange)o;
+      for (int d=0; d<this.min.length; ++d) {
+        if ((this.min[d] >= other.min[d] && this.max[d] <= other.max[d]) == false) {
+          // not within:
+          return false;
+        }
+      }
+      return true;
+    }
+
+    @Override
+    protected boolean contains(Range o) {
+      LongTestRange other = (LongTestRange) o;
+      for (int d=0; d<this.min.length; ++d) {
+        if ((this.min[d] <= other.min[d] && this.max[d] >= other.max[d]) == false) {
+          // not contains:
+          return false;
+        }
+      }
+      return true;
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder b = new StringBuilder();
+      b.append("Box(");
+      b.append(min[0]);
+      b.append(" TO ");
+      b.append(max[0]);
+      for (int d=1; d<min.length; ++d) {
+        b.append(", ");
+        b.append(min[d]);
+        b.append(" TO ");
+        b.append(max[d]);
+      }
+      b.append(")");
+
+      return b.toString();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/misc/src/java/org/apache/lucene/document/InetAddressPoint.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/document/InetAddressPoint.java b/lucene/misc/src/java/org/apache/lucene/document/InetAddressPoint.java
new file mode 100644
index 0000000..5cda742
--- /dev/null
+++ b/lucene/misc/src/java/org/apache/lucene/document/InetAddressPoint.java
@@ -0,0 +1,313 @@
+/*
+ * 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.document;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Arrays;
+import java.util.Comparator;
+
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.search.PointInSetQuery;
+import org.apache.lucene.search.PointRangeQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.NumericUtils;
+import org.apache.lucene.util.StringHelper;
+
+/** 
+ * An indexed 128-bit {@code InetAddress} field.
+ * <p>
+ * Finding all documents within a range at search time is
+ * efficient.  Multiple values for the same field in one document
+ * is allowed. 
+ * <p>
+ * This field defines static factory methods for creating common queries:
+ * <ul>
+ *   <li>{@link #newExactQuery(String, InetAddress)} for matching an exact network address.
+ *   <li>{@link #newPrefixQuery(String, InetAddress, int)} for matching a network based on CIDR prefix.
+ *   <li>{@link #newRangeQuery(String, InetAddress, InetAddress)} for matching arbitrary network address ranges.
+ *   <li>{@link #newSetQuery(String, InetAddress...)} for matching a set of network addresses.
+ * </ul>
+ * <p>
+ * This field supports both IPv4 and IPv6 addresses: IPv4 addresses are converted
+ * to <a href="https://tools.ietf.org/html/rfc4291#section-2.5.5">IPv4-Mapped IPv6 Addresses</a>:
+ * indexing {@code 1.2.3.4} is the same as indexing {@code ::FFFF:1.2.3.4}.
+ * @see PointValues
+ */
+public class InetAddressPoint extends Field {
+
+  // implementation note: we convert all addresses to IPv6: we expect prefix compression of values,
+  // so its not wasteful, but allows one field to handle both IPv4 and IPv6.
+  /** The number of bytes per dimension: 128 bits */
+  public static final int BYTES = 16;
+  
+  // rfc4291 prefix
+  static final byte[] IPV4_PREFIX = new byte[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, -1, -1 }; 
+
+  private static final FieldType TYPE;
+  static {
+    TYPE = new FieldType();
+    TYPE.setDimensions(1, BYTES);
+    TYPE.freeze();
+  }
+
+  /** The minimum value that an ip address can hold. */
+  public static final InetAddress MIN_VALUE;
+  /** The maximum value that an ip address can hold. */
+  public static final InetAddress MAX_VALUE;
+  static {
+    MIN_VALUE = decode(new byte[BYTES]);
+    byte[] maxValueBytes = new byte[BYTES];
+    Arrays.fill(maxValueBytes, (byte) 0xFF);
+    MAX_VALUE = decode(maxValueBytes);
+  }
+
+  /**
+   * Return the {@link InetAddress} that compares immediately greater than
+   * {@code address}.
+   * @throws ArithmeticException if the provided address is the
+   *              {@link #MAX_VALUE maximum ip address}
+   */
+  public static InetAddress nextUp(InetAddress address) {
+    if (address.equals(MAX_VALUE)) {
+      throw new ArithmeticException("Overflow: there is no greater InetAddress than "
+          + address.getHostAddress());
+    }
+    byte[] delta = new byte[BYTES];
+    delta[BYTES-1] = 1;
+    byte[] nextUpBytes = new byte[InetAddressPoint.BYTES];
+    NumericUtils.add(InetAddressPoint.BYTES, 0, encode(address), delta, nextUpBytes);
+    return decode(nextUpBytes);
+  }
+
+  /**
+   * Return the {@link InetAddress} that compares immediately less than
+   * {@code address}.
+   * @throws ArithmeticException if the provided address is the
+   *              {@link #MIN_VALUE minimum ip address}
+   */
+  public static InetAddress nextDown(InetAddress address) {
+    if (address.equals(MIN_VALUE)) {
+      throw new ArithmeticException("Underflow: there is no smaller InetAddress than "
+          + address.getHostAddress());
+    }
+    byte[] delta = new byte[BYTES];
+    delta[BYTES-1] = 1;
+    byte[] nextDownBytes = new byte[InetAddressPoint.BYTES];
+    NumericUtils.subtract(InetAddressPoint.BYTES, 0, encode(address), delta, nextDownBytes);
+    return decode(nextDownBytes);
+  }
+
+  /** Change the values of this field */
+  public void setInetAddressValue(InetAddress value) {
+    if (value == null) {
+      throw new IllegalArgumentException("point must not be null");
+    }
+    fieldsData = new BytesRef(encode(value));
+  }
+
+  @Override
+  public void setBytesValue(BytesRef bytes) {
+    throw new IllegalArgumentException("cannot change value type from InetAddress to BytesRef");
+  }
+
+  /** Creates a new InetAddressPoint, indexing the
+   *  provided address.
+   *
+   *  @param name field name
+   *  @param point InetAddress value
+   *  @throws IllegalArgumentException if the field name or value is null.
+   */
+  public InetAddressPoint(String name, InetAddress point) {
+    super(name, TYPE);
+    setInetAddressValue(point);
+  }
+  
+  @Override
+  public String toString() {
+    StringBuilder result = new StringBuilder();
+    result.append(getClass().getSimpleName());
+    result.append(" <");
+    result.append(name);
+    result.append(':');
+
+    // IPv6 addresses are bracketed, to not cause confusion with historic field:value representation
+    BytesRef bytes = (BytesRef) fieldsData;
+    InetAddress address = decode(BytesRef.deepCopyOf(bytes).bytes);
+    if (address.getAddress().length == 16) {
+      result.append('[');
+      result.append(address.getHostAddress());
+      result.append(']');
+    } else {
+      result.append(address.getHostAddress());
+    }
+
+    result.append('>');
+    return result.toString();
+  }
+  
+  // public helper methods (e.g. for queries)
+
+  /** Encode InetAddress value into binary encoding */
+  public static byte[] encode(InetAddress value) {
+    byte[] address = value.getAddress();
+    if (address.length == 4) {
+      byte[] mapped = new byte[16];
+      System.arraycopy(IPV4_PREFIX, 0, mapped, 0, IPV4_PREFIX.length);
+      System.arraycopy(address, 0, mapped, IPV4_PREFIX.length, address.length);
+      address = mapped;
+    } else if (address.length != 16) {
+      // more of an assertion, how did you create such an InetAddress :)
+      throw new UnsupportedOperationException("Only IPv4 and IPv6 addresses are supported");
+    }
+    return address;
+  }
+  
+  /** Decodes InetAddress value from binary encoding */
+  public static InetAddress decode(byte value[]) {
+    try {
+      return InetAddress.getByAddress(value);
+    } catch (UnknownHostException e) {
+      // this only happens if value.length != 4 or 16, strange exception class
+      throw new IllegalArgumentException("encoded bytes are of incorrect length", e);
+    }
+  }
+
+  // static methods for generating queries
+
+  /** 
+   * Create a query for matching a network address.
+   *
+   * @param field field name. must not be {@code null}.
+   * @param value exact value
+   * @throws IllegalArgumentException if {@code field} is null.
+   * @return a query matching documents with this exact value
+   */
+  public static Query newExactQuery(String field, InetAddress value) {
+    return newRangeQuery(field, value, value);
+  }
+  
+  /** 
+   * Create a prefix query for matching a CIDR network range.
+   *
+   * @param field field name. must not be {@code null}.
+   * @param value any host address
+   * @param prefixLength the network prefix length for this address. This is also known as the subnet mask in the context of IPv4 addresses.
+   * @throws IllegalArgumentException if {@code field} is null, or prefixLength is invalid.
+   * @return a query matching documents with addresses contained within this network
+   */
+  public static Query newPrefixQuery(String field, InetAddress value, int prefixLength) {
+    if (value == null) {
+      throw new IllegalArgumentException("InetAddress must not be null");
+    }
+    if (prefixLength < 0 || prefixLength > 8 * value.getAddress().length) {
+      throw new IllegalArgumentException("illegal prefixLength '" + prefixLength + "'. Must be 0-32 for IPv4 ranges, 0-128 for IPv6 ranges");
+    }
+    // create the lower value by zeroing out the host portion, upper value by filling it with all ones.
+    byte lower[] = value.getAddress();
+    byte upper[] = value.getAddress();
+    for (int i = prefixLength; i < 8 * lower.length; i++) {
+      int m = 1 << (7 - (i & 7));
+      lower[i >> 3] &= ~m;
+      upper[i >> 3] |= m;
+    }
+    try {
+      return newRangeQuery(field, InetAddress.getByAddress(lower), InetAddress.getByAddress(upper));
+    } catch (UnknownHostException e) {
+      throw new AssertionError(e); // values are coming from InetAddress
+    }
+  }
+
+  /** 
+   * Create a range query for network addresses.
+   * <p>
+   * You can have half-open ranges (which are in fact &lt;/&le; or &gt;/&ge; queries)
+   * by setting {@code lowerValue = InetAddressPoint.MIN_VALUE} or
+   * {@code upperValue = InetAddressPoint.MAX_VALUE}.
+   * <p> Ranges are inclusive. For exclusive ranges, pass {@code InetAddressPoint#nextUp(lowerValue)}
+   * or {@code InetAddressPoint#nexDown(upperValue)}.
+   *
+   * @param field field name. must not be {@code null}.
+   * @param lowerValue lower portion of the range (inclusive). must not be null.
+   * @param upperValue upper portion of the range (inclusive). must not be null.
+   * @throws IllegalArgumentException if {@code field} is null, {@code lowerValue} is null, 
+   *                                  or {@code upperValue} is null
+   * @return a query matching documents within this range.
+   */
+  public static Query newRangeQuery(String field, InetAddress lowerValue, InetAddress upperValue) {
+    PointRangeQuery.checkArgs(field, lowerValue, upperValue);
+    return new PointRangeQuery(field, encode(lowerValue), encode(upperValue), 1) {
+      @Override
+      protected String toString(int dimension, byte[] value) {
+        return decode(value).getHostAddress(); // for ranges, the range itself is already bracketed
+      }
+    };
+  }
+
+  /**
+   * Create a query matching any of the specified 1D values.  This is the points equivalent of {@code TermsQuery}.
+   * 
+   * @param field field name. must not be {@code null}.
+   * @param values all values to match
+   */
+  public static Query newSetQuery(String field, InetAddress... values) {
+
+    // We must compare the encoded form (InetAddress doesn't implement Comparable, and even if it
+    // did, we do our own thing with ipv4 addresses):
+
+    // NOTE: we could instead convert-per-comparison and save this extra array, at cost of slower sort:
+    byte[][] sortedValues = new byte[values.length][];
+    for(int i=0;i<values.length;i++) {
+      sortedValues[i] = encode(values[i]);
+    }
+
+    Arrays.sort(sortedValues,
+                new Comparator<byte[]>() {
+                  @Override
+                  public int compare(byte[] a, byte[] b) {
+                    return StringHelper.compare(BYTES, a, 0, b, 0);
+                  }
+                });
+
+    final BytesRef encoded = new BytesRef(new byte[BYTES]);
+
+    return new PointInSetQuery(field, 1, BYTES,
+                               new PointInSetQuery.Stream() {
+
+                                 int upto;
+
+                                 @Override
+                                 public BytesRef next() {
+                                   if (upto == sortedValues.length) {
+                                     return null;
+                                   } else {
+                                     encoded.bytes = sortedValues[upto];
+                                     assert encoded.bytes.length == encoded.length;
+                                     upto++;
+                                     return encoded;
+                                   }
+                                 }
+                               }) {
+      @Override
+      protected String toString(byte[] value) {
+        assert value.length == BYTES;
+        return decode(value).getHostAddress();
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/misc/src/java/org/apache/lucene/document/InetAddressRange.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/document/InetAddressRange.java b/lucene/misc/src/java/org/apache/lucene/document/InetAddressRange.java
new file mode 100644
index 0000000..5fa1fb9
--- /dev/null
+++ b/lucene/misc/src/java/org/apache/lucene/document/InetAddressRange.java
@@ -0,0 +1,168 @@
+/*
+ * 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.document;
+
+import java.net.InetAddress;
+
+import org.apache.lucene.document.RangeFieldQuery.QueryType;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.StringHelper;
+
+/**
+ * An indexed InetAddress Range Field
+ * <p>
+ * This field indexes an {@code InetAddress} range defined as a min/max pairs. It is single
+ * dimension only (indexed as two 16 byte paired values).
+ * <p>
+ * Multiple values are supported.
+ *
+ * <p>
+ * This field defines the following static factory methods for common search operations over Ip Ranges
+ * <ul>
+ *   <li>{@link #newIntersectsQuery newIntersectsQuery()} matches ip ranges that intersect the defined search range.
+ *   <li>{@link #newWithinQuery newWithinQuery()} matches ip ranges that are within the defined search range.
+ *   <li>{@link #newContainsQuery newContainsQuery()} matches ip ranges that contain the defined search range.
+ *   <li>{@link #newCrossesQuery newCrossesQuery()} matches ip ranges that cross the defined search range
+ * </ul>
+ */
+public class InetAddressRange extends Field {
+  /** The number of bytes per dimension : sync w/ {@code InetAddressPoint} */
+  public static final int BYTES = InetAddressPoint.BYTES;
+
+  private static final FieldType TYPE;
+  static {
+    TYPE = new FieldType();
+    TYPE.setDimensions(2, BYTES);
+    TYPE.freeze();
+  }
+
+  /**
+   * Create a new InetAddressRange from min/max value
+   * @param name field name. must not be null.
+   * @param min range min value; defined as an {@code InetAddress}
+   * @param max range max value; defined as an {@code InetAddress}
+   */
+  public InetAddressRange(String name, final InetAddress min, final InetAddress max) {
+    super(name, TYPE);
+    setRangeValues(min, max);
+  }
+
+  /**
+   * Change (or set) the min/max values of the field.
+   * @param min range min value; defined as an {@code InetAddress}
+   * @param max range max value; defined as an {@code InetAddress}
+   */
+  public void setRangeValues(InetAddress min, InetAddress max) {
+    if (StringHelper.compare(BYTES, min.getAddress(), 0, max.getAddress(), 0) > 0) {
+      throw new IllegalArgumentException("min value cannot be greater than max value for range field (name=" + name + ")");
+    }
+    final byte[] bytes;
+    if (fieldsData == null) {
+      bytes = new byte[BYTES*2];
+      fieldsData = new BytesRef(bytes);
+    } else {
+      bytes = ((BytesRef)fieldsData).bytes;
+    }
+    encode(min, max, bytes);
+  }
+
+  /** encode the min/max range into the provided byte array */
+  private static void encode(final InetAddress min, final InetAddress max, final byte[] bytes) {
+    System.arraycopy(InetAddressPoint.encode(min), 0, bytes, 0, BYTES);
+    System.arraycopy(InetAddressPoint.encode(max), 0, bytes, BYTES, BYTES);
+  }
+
+  /** encode the min/max range and return the byte array */
+  private static byte[] encode(InetAddress min, InetAddress max) {
+    byte[] b = new byte[BYTES*2];
+    encode(min, max, b);
+    return b;
+  }
+
+  /**
+   * Create a query for matching indexed ip ranges that {@code INTERSECT} the defined range.
+   * @param field field name. must not be null.
+   * @param min range min value; provided as an {@code InetAddress}
+   * @param max range max value; provided as an {@code InetAddress}
+   * @return query for matching intersecting ranges (overlap, within, crosses, or contains)
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newIntersectsQuery(String field, final InetAddress min, final InetAddress max) {
+    return newRelationQuery(field, min, max, QueryType.INTERSECTS);
+  }
+
+  /**
+   * Create a query for matching indexed ip ranges that {@code CONTAINS} the defined range.
+   * @param field field name. must not be null.
+   * @param min range min value; provided as an {@code InetAddress}
+   * @param max range max value; provided as an {@code InetAddress}
+   * @return query for matching intersecting ranges (overlap, within, crosses, or contains)
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newContainsQuery(String field, final InetAddress min, final InetAddress max) {
+    return newRelationQuery(field, min, max, QueryType.CONTAINS);
+  }
+
+  /**
+   * Create a query for matching indexed ip ranges that are {@code WITHIN} the defined range.
+   * @param field field name. must not be null.
+   * @param min range min value; provided as an {@code InetAddress}
+   * @param max range max value; provided as an {@code InetAddress}
+   * @return query for matching intersecting ranges (overlap, within, crosses, or contains)
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newWithinQuery(String field, final InetAddress min, final InetAddress max) {
+    return newRelationQuery(field, min, max, QueryType.WITHIN);
+  }
+
+  /**
+   * Create a query for matching indexed ip ranges that {@code CROSS} the defined range.
+   * @param field field name. must not be null.
+   * @param min range min value; provided as an {@code InetAddress}
+   * @param max range max value; provided as an {@code InetAddress}
+   * @return query for matching intersecting ranges (overlap, within, crosses, or contains)
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newCrossesQuery(String field, final InetAddress min, final InetAddress max) {
+    return newRelationQuery(field, min, max, QueryType.CROSSES);
+  }
+
+  /** helper method for creating the desired relational query */
+  private static Query newRelationQuery(String field, final InetAddress min, final InetAddress max, QueryType relation) {
+    return new RangeFieldQuery(field, encode(min, max), 1, relation) {
+      @Override
+      protected String toString(byte[] ranges, int dimension) {
+        return InetAddressRange.toString(ranges, dimension);
+      }
+    };
+  }
+
+  /**
+   * Returns the String representation for the range at the given dimension
+   * @param ranges the encoded ranges, never null
+   * @param dimension the dimension of interest (not used for this field)
+   * @return The string representation for the range at the provided dimension
+   */
+  private static String toString(byte[] ranges, int dimension) {
+    byte[] min = new byte[BYTES];
+    System.arraycopy(ranges, 0, min, 0, BYTES);
+    byte[] max = new byte[BYTES];
+    System.arraycopy(ranges, BYTES, max, 0, BYTES);
+    return "[" + InetAddressPoint.decode(min) + " : " + InetAddressPoint.decode(max) + "]";
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/misc/src/test/org/apache/lucene/document/TestInetAddressPoint.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/test/org/apache/lucene/document/TestInetAddressPoint.java b/lucene/misc/src/test/org/apache/lucene/document/TestInetAddressPoint.java
new file mode 100644
index 0000000..0e0901b
--- /dev/null
+++ b/lucene/misc/src/test/org/apache/lucene/document/TestInetAddressPoint.java
@@ -0,0 +1,176 @@
+/*
+ * 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.document;
+
+import java.net.InetAddress;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+
+/** Simple tests for {@link InetAddressPoint} */
+public class TestInetAddressPoint extends LuceneTestCase {
+
+  /** Add a single address and search for it */
+  public void testBasics() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+
+    // add a doc with an address
+    Document document = new Document();
+    InetAddress address = InetAddress.getByName("1.2.3.4");
+    document.add(new InetAddressPoint("field", address));
+    writer.addDocument(document);
+    
+    // search and verify we found our doc
+    IndexReader reader = writer.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+    assertEquals(1, searcher.count(InetAddressPoint.newExactQuery("field", address)));
+    assertEquals(1, searcher.count(InetAddressPoint.newPrefixQuery("field", address, 24)));
+    assertEquals(1, searcher.count(InetAddressPoint.newRangeQuery("field", InetAddress.getByName("1.2.3.3"), InetAddress.getByName("1.2.3.5"))));
+    assertEquals(1, searcher.count(InetAddressPoint.newSetQuery("field", InetAddress.getByName("1.2.3.4"))));
+    assertEquals(1, searcher.count(InetAddressPoint.newSetQuery("field", InetAddress.getByName("1.2.3.4"), InetAddress.getByName("1.2.3.5"))));
+    assertEquals(0, searcher.count(InetAddressPoint.newSetQuery("field", InetAddress.getByName("1.2.3.3"))));
+    assertEquals(0, searcher.count(InetAddressPoint.newSetQuery("field")));
+
+    reader.close();
+    writer.close();
+    dir.close();
+  }
+  
+  /** Add a single address and search for it */
+  public void testBasicsV6() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+
+    // add a doc with an address
+    Document document = new Document();
+    InetAddress address = InetAddress.getByName("fec0::f66d");
+    document.add(new InetAddressPoint("field", address));
+    writer.addDocument(document);
+    
+    // search and verify we found our doc
+    IndexReader reader = writer.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+    assertEquals(1, searcher.count(InetAddressPoint.newExactQuery("field", address)));
+    assertEquals(1, searcher.count(InetAddressPoint.newPrefixQuery("field", address, 64)));
+    assertEquals(1, searcher.count(InetAddressPoint.newRangeQuery("field", InetAddress.getByName("fec0::f66c"), InetAddress.getByName("fec0::f66e"))));
+
+    reader.close();
+    writer.close();
+    dir.close();
+  }
+    
+  public void testToString() throws Exception {
+    assertEquals("InetAddressPoint <field:1.2.3.4>", new InetAddressPoint("field", InetAddress.getByName("1.2.3.4")).toString());
+    assertEquals("InetAddressPoint <field:1.2.3.4>", new InetAddressPoint("field", InetAddress.getByName("::FFFF:1.2.3.4")).toString());
+    assertEquals("InetAddressPoint <field:[fdc8:57ed:f042:ad1:f66d:4ff:fe90:ce0c]>", new InetAddressPoint("field", InetAddress.getByName("fdc8:57ed:f042:0ad1:f66d:4ff:fe90:ce0c")).toString());
+    
+    assertEquals("field:[1.2.3.4 TO 1.2.3.4]", InetAddressPoint.newExactQuery("field", InetAddress.getByName("1.2.3.4")).toString());
+    assertEquals("field:[0:0:0:0:0:0:0:1 TO 0:0:0:0:0:0:0:1]", InetAddressPoint.newExactQuery("field", InetAddress.getByName("::1")).toString());
+    
+    assertEquals("field:[1.2.3.0 TO 1.2.3.255]", InetAddressPoint.newPrefixQuery("field", InetAddress.getByName("1.2.3.4"), 24).toString());
+    assertEquals("field:[fdc8:57ed:f042:ad1:0:0:0:0 TO fdc8:57ed:f042:ad1:ffff:ffff:ffff:ffff]", InetAddressPoint.newPrefixQuery("field", InetAddress.getByName("fdc8:57ed:f042:0ad1:f66d:4ff:fe90:ce0c"), 64).toString());
+    assertEquals("field:{fdc8:57ed:f042:ad1:f66d:4ff:fe90:ce0c}", InetAddressPoint.newSetQuery("field", InetAddress.getByName("fdc8:57ed:f042:0ad1:f66d:4ff:fe90:ce0c")).toString());
+  }
+
+  public void testQueryEquals() throws Exception {
+    Query q1, q2;
+    q1 = InetAddressPoint.newRangeQuery("a", InetAddress.getByName("1.2.3.3"), InetAddress.getByName("1.2.3.5"));
+    q2 = InetAddressPoint.newRangeQuery("a", InetAddress.getByName("1.2.3.3"), InetAddress.getByName("1.2.3.5"));
+    assertEquals(q1, q2);
+    assertEquals(q1.hashCode(), q2.hashCode());
+    assertFalse(q1.equals(InetAddressPoint.newRangeQuery("a", InetAddress.getByName("1.2.3.3"), InetAddress.getByName("1.2.3.7"))));
+    assertFalse(q1.equals(InetAddressPoint.newRangeQuery("b", InetAddress.getByName("1.2.3.3"), InetAddress.getByName("1.2.3.5"))));
+
+    q1 = InetAddressPoint.newPrefixQuery("a", InetAddress.getByName("1.2.3.3"), 16);
+    q2 = InetAddressPoint.newPrefixQuery("a", InetAddress.getByName("1.2.3.3"), 16);
+    assertEquals(q1, q2);
+    assertEquals(q1.hashCode(), q2.hashCode());
+    assertFalse(q1.equals(InetAddressPoint.newPrefixQuery("a", InetAddress.getByName("1.1.3.5"), 16)));
+    assertFalse(q1.equals(InetAddressPoint.newPrefixQuery("a", InetAddress.getByName("1.2.3.5"), 24)));
+
+    q1 = InetAddressPoint.newExactQuery("a", InetAddress.getByName("1.2.3.3"));
+    q2 = InetAddressPoint.newExactQuery("a", InetAddress.getByName("1.2.3.3"));
+    assertEquals(q1, q2);
+    assertEquals(q1.hashCode(), q2.hashCode());
+    assertFalse(q1.equals(InetAddressPoint.newExactQuery("a", InetAddress.getByName("1.2.3.5"))));
+
+    q1 = InetAddressPoint.newSetQuery("a", InetAddress.getByName("1.2.3.3"), InetAddress.getByName("1.2.3.5"));
+    q2 = InetAddressPoint.newSetQuery("a", InetAddress.getByName("1.2.3.3"), InetAddress.getByName("1.2.3.5"));
+    assertEquals(q1, q2);
+    assertEquals(q1.hashCode(), q2.hashCode());
+    assertFalse(q1.equals(InetAddressPoint.newSetQuery("a", InetAddress.getByName("1.2.3.3"), InetAddress.getByName("1.2.3.7"))));
+  }
+
+  public void testPrefixQuery() throws Exception {
+    assertEquals(
+        InetAddressPoint.newRangeQuery("a", InetAddress.getByName("1.2.3.0"), InetAddress.getByName("1.2.3.255")),
+        InetAddressPoint.newPrefixQuery("a", InetAddress.getByName("1.2.3.127"), 24));
+    assertEquals(
+        InetAddressPoint.newRangeQuery("a", InetAddress.getByName("1.2.3.128"), InetAddress.getByName("1.2.3.255")),
+        InetAddressPoint.newPrefixQuery("a", InetAddress.getByName("1.2.3.213"), 25));
+    assertEquals(
+        InetAddressPoint.newRangeQuery("a", InetAddress.getByName("2001::a000:0"), InetAddress.getByName("2001::afff:ffff")),
+        InetAddressPoint.newPrefixQuery("a", InetAddress.getByName("2001::a6bd:fc80"), 100));
+  }
+
+  public void testNextUp() throws Exception {
+    assertEquals(InetAddress.getByName("::1"),
+        InetAddressPoint.nextUp(InetAddress.getByName("::")));
+
+    assertEquals(InetAddress.getByName("::1:0"),
+        InetAddressPoint.nextUp(InetAddress.getByName("::ffff")));
+
+    assertEquals(InetAddress.getByName("1.2.4.0"),
+        InetAddressPoint.nextUp(InetAddress.getByName("1.2.3.255")));
+
+    assertEquals(InetAddress.getByName("0.0.0.0"),
+        InetAddressPoint.nextUp(InetAddress.getByName("::fffe:ffff:ffff")));
+
+    assertEquals(InetAddress.getByName("::1:0:0:0"),
+        InetAddressPoint.nextUp(InetAddress.getByName("255.255.255.255")));
+
+    ArithmeticException e = expectThrows(ArithmeticException.class,
+        () -> InetAddressPoint.nextUp(InetAddress.getByName("ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff")));
+    assertEquals("Overflow: there is no greater InetAddress than ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff", e.getMessage());
+  }
+
+  public void testNextDown() throws Exception {
+    assertEquals(InetAddress.getByName("ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffe"),
+        InetAddressPoint.nextDown(InetAddress.getByName("ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff")));
+
+    assertEquals(InetAddress.getByName("::ffff"),
+        InetAddressPoint.nextDown(InetAddress.getByName("::1:0")));
+
+    assertEquals(InetAddress.getByName("1.2.3.255"),
+        InetAddressPoint.nextDown(InetAddress.getByName("1.2.4.0")));
+
+    assertEquals(InetAddress.getByName("::fffe:ffff:ffff"),
+        InetAddressPoint.nextDown(InetAddress.getByName("0.0.0.0")));
+
+    assertEquals(InetAddress.getByName("255.255.255.255"),
+        InetAddressPoint.nextDown(InetAddress.getByName("::1:0:0:0")));
+
+    ArithmeticException e = expectThrows(ArithmeticException.class,
+        () -> InetAddressPoint.nextDown(InetAddress.getByName("::")));
+    assertEquals("Underflow: there is no smaller InetAddress than 0:0:0:0:0:0:0:0", e.getMessage());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/misc/src/test/org/apache/lucene/search/TestInetAddressRangeQueries.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/test/org/apache/lucene/search/TestInetAddressRangeQueries.java b/lucene/misc/src/test/org/apache/lucene/search/TestInetAddressRangeQueries.java
new file mode 100644
index 0000000..e22cf9b
--- /dev/null
+++ b/lucene/misc/src/test/org/apache/lucene/search/TestInetAddressRangeQueries.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.lucene.search;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+
+import org.apache.lucene.document.InetAddressRange;
+import org.apache.lucene.util.StringHelper;
+
+/**
+ * Random testing for {@link InetAddressRange}
+ */
+public class TestInetAddressRangeQueries extends BaseRangeFieldQueryTestCase {
+  private static final String FIELD_NAME = "ipRangeField";
+
+  private IPVersion ipVersion;
+
+  private enum IPVersion {IPv4, IPv6}
+
+  @Override
+  protected Range nextRange(int dimensions) throws Exception {
+    InetAddress min = nextInetaddress();
+    byte[] bMin = min.getAddress();
+    InetAddress max = nextInetaddress();
+    byte[] bMax = max.getAddress();
+    if (StringHelper.compare(bMin.length, bMin, 0, bMax, 0) > 0) {
+      return new IpRange(max, min);
+    }
+    return new IpRange(min, max);
+  }
+
+  /** return random IPv4 or IPv6 address */
+  private InetAddress nextInetaddress() throws UnknownHostException {
+    byte[] b;
+    switch (ipVersion) {
+      case IPv4:
+        b = new byte[4];
+        break;
+      case IPv6:
+        b = new byte[16];
+        break;
+      default:
+        throw new IllegalArgumentException("incorrect IP version: " + ipVersion);
+    }
+    random().nextBytes(b);
+    return InetAddress.getByAddress(b);
+  }
+
+  /** randomly select version across tests */
+  private IPVersion ipVersion() {
+    return random().nextBoolean() ? IPVersion.IPv4 : IPVersion.IPv6;
+  }
+
+  @Override
+  public void testRandomTiny() throws Exception {
+    ipVersion = ipVersion();
+    super.testRandomTiny();
+  }
+
+  @Override
+  public void testMultiValued() throws Exception {
+    ipVersion = ipVersion();
+    super.testRandomMedium();
+  }
+
+  @Override
+  public void testRandomMedium() throws Exception {
+    ipVersion = ipVersion();
+    super.testMultiValued();
+  }
+
+  @Nightly
+  @Override
+  public void testRandomBig() throws Exception {
+    ipVersion = ipVersion();
+    super.testRandomBig();
+  }
+
+  /** return random range */
+  @Override
+  protected InetAddressRange newRangeField(Range r) {
+    return new InetAddressRange(FIELD_NAME, ((IpRange)r).min, ((IpRange)r).max);
+  }
+
+  /** return random intersects query */
+  @Override
+  protected Query newIntersectsQuery(Range r) {
+    return InetAddressRange.newIntersectsQuery(FIELD_NAME, ((IpRange)r).min, ((IpRange)r).max);
+  }
+
+  /** return random contains query */
+  @Override
+  protected Query newContainsQuery(Range r) {
+    return InetAddressRange.newContainsQuery(FIELD_NAME, ((IpRange)r).min, ((IpRange)r).max);
+  }
+
+  /** return random within query */
+  @Override
+  protected Query newWithinQuery(Range r) {
+    return InetAddressRange.newWithinQuery(FIELD_NAME, ((IpRange)r).min, ((IpRange)r).max);
+  }
+
+  /** return random crosses query */
+  @Override
+  protected Query newCrossesQuery(Range r) {
+    return InetAddressRange.newCrossesQuery(FIELD_NAME, ((IpRange)r).min, ((IpRange)r).max);
+  }
+
+  /** encapsulated IpRange for test validation */
+  private class IpRange extends Range {
+    InetAddress min;
+    InetAddress max;
+
+    IpRange(InetAddress min, InetAddress max) {
+      this.min = min;
+      this.max = max;
+    }
+
+    @Override
+    protected int numDimensions() {
+      return 1;
+    }
+
+    @Override
+    protected InetAddress getMin(int dim) {
+      return min;
+    }
+
+    @Override
+    protected void setMin(int dim, Object val) {
+      byte[] v = ((InetAddress)val).getAddress();
+
+      if (StringHelper.compare(v.length, min.getAddress(), 0, v, 0) < 0) {
+        max = (InetAddress)val;
+      } else {
+        min = (InetAddress) val;
+      }
+    }
+
+    @Override
+    protected InetAddress getMax(int dim) {
+      return max;
+    }
+
+    @Override
+    protected void setMax(int dim, Object val) {
+      byte[] v = ((InetAddress)val).getAddress();
+
+      if (StringHelper.compare(v.length, max.getAddress(), 0, v, 0) > 0) {
+        min = (InetAddress)val;
+      } else {
+        max = (InetAddress) val;
+      }
+    }
+
+    @Override
+    protected boolean isEqual(Range o) {
+      IpRange other = (IpRange)o;
+      return this.min.equals(other.min) && this.max.equals(other.max);
+    }
+
+    @Override
+    protected boolean isDisjoint(Range o) {
+      IpRange other = (IpRange)o;
+      byte[] bMin = min.getAddress();
+      byte[] bMax = max.getAddress();
+      return StringHelper.compare(bMin.length, bMin, 0, other.max.getAddress(), 0) > 0 ||
+          StringHelper.compare(bMax.length, bMax, 0, other.min.getAddress(), 0) < 0;
+    }
+
+    @Override
+    protected boolean isWithin(Range o) {
+      IpRange other = (IpRange)o;
+      byte[] bMin = min.getAddress();
+      byte[] bMax = max.getAddress();
+      return StringHelper.compare(bMin.length, bMin, 0, other.min.getAddress(), 0) >= 0 &&
+          StringHelper.compare(bMax.length, bMax, 0, other.max.getAddress(), 0) <= 0;
+    }
+
+    @Override
+    protected boolean contains(Range o) {
+      IpRange other = (IpRange)o;
+      byte[] bMin = min.getAddress();
+      byte[] bMax = max.getAddress();
+      return StringHelper.compare(bMin.length, bMin, 0, other.min.getAddress(), 0) <= 0 &&
+          StringHelper.compare(bMax.length, bMax, 0, other.max.getAddress(), 0) >= 0;
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder b = new StringBuilder();
+      b.append("Box(");
+      b.append(min.getHostAddress());
+      b.append(" TO ");
+      b.append(max.getHostAddress());
+      b.append(")");
+      return b.toString();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/sandbox/src/java/org/apache/lucene/document/DoubleRangeField.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/DoubleRangeField.java b/lucene/sandbox/src/java/org/apache/lucene/document/DoubleRangeField.java
deleted file mode 100644
index c5ae0e7..0000000
--- a/lucene/sandbox/src/java/org/apache/lucene/document/DoubleRangeField.java
+++ /dev/null
@@ -1,282 +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.lucene.document;
-
-import org.apache.lucene.document.RangeFieldQuery.QueryType;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.NumericUtils;
-
-/**
- * An indexed Double Range field.
- * <p>
- * This field indexes dimensional ranges defined as min/max pairs. It supports
- * up to a maximum of 4 dimensions (indexed as 8 numeric values). With 1 dimension representing a single double range,
- * 2 dimensions representing a bounding box, 3 dimensions a bounding cube, and 4 dimensions a tesseract.
- * <p>
- * Multiple values for the same field in one document is supported, and open ended ranges can be defined using
- * {@code Double.NEGATIVE_INFINITY} and {@code Double.POSITIVE_INFINITY}.
- *
- * <p>
- * This field defines the following static factory methods for common search operations over double ranges:
- * <ul>
- *   <li>{@link #newIntersectsQuery newIntersectsQuery()} matches ranges that intersect the defined search range.
- *   <li>{@link #newWithinQuery newWithinQuery()} matches ranges that are within the defined search range.
- *   <li>{@link #newContainsQuery newContainsQuery()} matches ranges that contain the defined search range.
- * </ul>
- */
-public class DoubleRangeField extends Field {
-  /** stores double values so number of bytes is 8 */
-  public static final int BYTES = Double.BYTES;
-
-  /**
-   * Create a new DoubleRangeField type, from min/max parallel arrays
-   *
-   * @param name field name. must not be null.
-   * @param min range min values; each entry is the min value for the dimension
-   * @param max range max values; each entry is the max value for the dimension
-   */
-  public DoubleRangeField(String name, final double[] min, final double[] max) {
-    super(name, getType(min.length));
-    setRangeValues(min, max);
-  }
-
-  /** set the field type */
-  private static FieldType getType(int dimensions) {
-    if (dimensions > 4) {
-      throw new IllegalArgumentException("DoubleRangeField does not support greater than 4 dimensions");
-    }
-
-    FieldType ft = new FieldType();
-    // dimensions is set as 2*dimension size (min/max per dimension)
-    ft.setDimensions(dimensions*2, BYTES);
-    ft.freeze();
-    return ft;
-  }
-
-  /**
-   * Changes the values of the field.
-   * @param min array of min values. (accepts {@code Double.NEGATIVE_INFINITY})
-   * @param max array of max values. (accepts {@code Double.POSITIVE_INFINITY})
-   * @throws IllegalArgumentException if {@code min} or {@code max} is invalid
-   */
-  public void setRangeValues(double[] min, double[] max) {
-    checkArgs(min, max);
-    if (min.length*2 != type.pointDimensionCount() || max.length*2 != type.pointDimensionCount()) {
-      throw new IllegalArgumentException("field (name=" + name + ") uses " + type.pointDimensionCount()/2
-          + " dimensions; cannot change to (incoming) " + min.length + " dimensions");
-    }
-
-    final byte[] bytes;
-    if (fieldsData == null) {
-      bytes = new byte[BYTES*2*min.length];
-      fieldsData = new BytesRef(bytes);
-    } else {
-      bytes = ((BytesRef)fieldsData).bytes;
-    }
-    verifyAndEncode(min, max, bytes);
-  }
-
-  /** validate the arguments */
-  private static void checkArgs(final double[] min, final double[] max) {
-    if (min == null || max == null || min.length == 0 || max.length == 0) {
-      throw new IllegalArgumentException("min/max range values cannot be null or empty");
-    }
-    if (min.length != max.length) {
-      throw new IllegalArgumentException("min/max ranges must agree");
-    }
-    if (min.length > 4) {
-      throw new IllegalArgumentException("DoubleRangeField does not support greater than 4 dimensions");
-    }
-  }
-
-  /**
-   * Encodes the min, max ranges into a byte array
-   */
-  private static byte[] encode(double[] min, double[] max) {
-    checkArgs(min, max);
-    byte[] b = new byte[BYTES*2*min.length];
-    verifyAndEncode(min, max, b);
-    return b;
-  }
-
-  /**
-   * encode the ranges into a sortable byte array ({@code Double.NaN} not allowed)
-   * <p>
-   * example for 4 dimensions (8 bytes per dimension value):
-   * minD1 ... minD4 | maxD1 ... maxD4
-   */
-  static void verifyAndEncode(double[] min, double[] max, byte[] bytes) {
-    for (int d=0,i=0,j=min.length*BYTES; d<min.length; ++d, i+=BYTES, j+=BYTES) {
-      if (Double.isNaN(min[d])) {
-        throw new IllegalArgumentException("invalid min value (" + Double.NaN + ")" + " in DoubleRangeField");
-      }
-      if (Double.isNaN(max[d])) {
-        throw new IllegalArgumentException("invalid max value (" + Double.NaN + ")" + " in DoubleRangeField");
-      }
-      if (min[d] > max[d]) {
-        throw new IllegalArgumentException("min value (" + min[d] + ") is greater than max value (" + max[d] + ")");
-      }
-      encode(min[d], bytes, i);
-      encode(max[d], bytes, j);
-    }
-  }
-
-  /** encode the given value into the byte array at the defined offset */
-  private static void encode(double val, byte[] bytes, int offset) {
-    NumericUtils.longToSortableBytes(NumericUtils.doubleToSortableLong(val), bytes, offset);
-  }
-
-  /**
-   * Get the min value for the given dimension
-   * @param dimension the dimension, always positive
-   * @return the decoded min value
-   */
-  public double getMin(int dimension) {
-    if (dimension < 0 || dimension >= type.pointDimensionCount()/2) {
-      throw new IllegalArgumentException("dimension request (" + dimension +
-          ") out of bounds for field (name=" + name + " dimensions=" + type.pointDimensionCount()/2 + "). ");
-    }
-    return decodeMin(((BytesRef)fieldsData).bytes, dimension);
-  }
-
-  /**
-   * Get the max value for the given dimension
-   * @param dimension the dimension, always positive
-   * @return the decoded max value
-   */
-  public double getMax(int dimension) {
-    if (dimension < 0 || dimension >= type.pointDimensionCount()/2) {
-      throw new IllegalArgumentException("dimension request (" + dimension +
-          ") out of bounds for field (name=" + name + " dimensions=" + type.pointDimensionCount()/2 + "). ");
-    }
-    return decodeMax(((BytesRef)fieldsData).bytes, dimension);
-  }
-
-  /** decodes the min value (for the defined dimension) from the encoded input byte array */
-  static double decodeMin(byte[] b, int dimension) {
-    int offset = dimension*BYTES;
-    return NumericUtils.sortableLongToDouble(NumericUtils.sortableBytesToLong(b, offset));
-  }
-
-  /** decodes the max value (for the defined dimension) from the encoded input byte array */
-  static double decodeMax(byte[] b, int dimension) {
-    int offset = b.length/2 + dimension*BYTES;
-    return NumericUtils.sortableLongToDouble(NumericUtils.sortableBytesToLong(b, offset));
-  }
-
-  /**
-   * Create a query for matching indexed ranges that intersect the defined range.
-   * @param field field name. must not be null.
-   * @param min array of min values. (accepts {@code Double.NEGATIVE_INFINITY})
-   * @param max array of max values. (accepts {@code Double.POSITIVE_INFINITY})
-   * @return query for matching intersecting ranges (overlap, within, or contains)
-   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
-   */
-  public static Query newIntersectsQuery(String field, final double[] min, final double[] max) {
-    return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.INTERSECTS) {
-      @Override
-      protected String toString(byte[] ranges, int dimension) {
-        return DoubleRangeField.toString(ranges, dimension);
-      }
-    };
-  }
-
-  /**
-   * Create a query for matching indexed ranges that contain the defined range.
-   * @param field field name. must not be null.
-   * @param min array of min values. (accepts {@code Double.MIN_VALUE})
-   * @param max array of max values. (accepts {@code Double.MAX_VALUE})
-   * @return query for matching ranges that contain the defined range
-   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
-   */
-  public static Query newContainsQuery(String field, final double[] min, final double[] max) {
-    return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.CONTAINS) {
-      @Override
-      protected String toString(byte[] ranges, int dimension) {
-        return DoubleRangeField.toString(ranges, dimension);
-      }
-    };
-  }
-
-  /**
-   * Create a query for matching indexed ranges that are within the defined range.
-   * @param field field name. must not be null.
-   * @param min array of min values. (accepts {@code Double.MIN_VALUE})
-   * @param max array of max values. (accepts {@code Double.MAX_VALUE})
-   * @return query for matching ranges within the defined range
-   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
-   */
-  public static Query newWithinQuery(String field, final double[] min, final double[] max) {
-    checkArgs(min, max);
-    return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.WITHIN) {
-      @Override
-      protected String toString(byte[] ranges, int dimension) {
-        return DoubleRangeField.toString(ranges, dimension);
-      }
-    };
-  }
-
-  /**
-   * Create a query for matching indexed ranges that cross the defined range.
-   * A CROSSES is defined as any set of ranges that are not disjoint and not wholly contained by
-   * the query. Effectively, its the complement of union(WITHIN, DISJOINT).
-   * @param field field name. must not be null.
-   * @param min array of min values. (accepts {@code Double.MIN_VALUE})
-   * @param max array of max values. (accepts {@code Double.MAX_VALUE})
-   * @return query for matching ranges within the defined range
-   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
-   */
-  public static Query newCrossesQuery(String field, final double[] min, final double[] max) {
-    checkArgs(min, max);
-    return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.CROSSES) {
-      @Override
-      protected String toString(byte[] ranges, int dimension) {
-        return DoubleRangeField.toString(ranges, dimension);
-      }
-    };
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder();
-    sb.append(getClass().getSimpleName());
-    sb.append(" <");
-    sb.append(name);
-    sb.append(':');
-    byte[] b = ((BytesRef)fieldsData).bytes;
-    toString(b, 0);
-    for (int d=1; d<type.pointDimensionCount(); ++d) {
-      sb.append(' ');
-      toString(b, d);
-    }
-    sb.append('>');
-
-    return sb.toString();
-  }
-
-  /**
-   * Returns the String representation for the range at the given dimension
-   * @param ranges the encoded ranges, never null
-   * @param dimension the dimension of interest
-   * @return The string representation for the range at the provided dimension
-   */
-  private static String toString(byte[] ranges, int dimension) {
-    return "[" + Double.toString(decodeMin(ranges, dimension)) + " : "
-        + Double.toString(decodeMax(ranges, dimension)) + "]";
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/sandbox/src/java/org/apache/lucene/document/FloatRangeField.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/FloatRangeField.java b/lucene/sandbox/src/java/org/apache/lucene/document/FloatRangeField.java
deleted file mode 100644
index 60e0acf..0000000
--- a/lucene/sandbox/src/java/org/apache/lucene/document/FloatRangeField.java
+++ /dev/null
@@ -1,282 +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.lucene.document;
-
-import org.apache.lucene.document.RangeFieldQuery.QueryType;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.NumericUtils;
-
-/**
- * An indexed Float Range field.
- * <p>
- * This field indexes dimensional ranges defined as min/max pairs. It supports
- * up to a maximum of 4 dimensions (indexed as 8 numeric values). With 1 dimension representing a single float range,
- * 2 dimensions representing a bounding box, 3 dimensions a bounding cube, and 4 dimensions a tesseract.
- * <p>
- * Multiple values for the same field in one document is supported, and open ended ranges can be defined using
- * {@code Float.NEGATIVE_INFINITY} and {@code Float.POSITIVE_INFINITY}.
- *
- * <p>
- * This field defines the following static factory methods for common search operations over float ranges:
- * <ul>
- *   <li>{@link #newIntersectsQuery newIntersectsQuery()} matches ranges that intersect the defined search range.
- *   <li>{@link #newWithinQuery newWithinQuery()} matches ranges that are within the defined search range.
- *   <li>{@link #newContainsQuery newContainsQuery()} matches ranges that contain the defined search range.
- * </ul>
- */
-public class FloatRangeField extends Field {
-  /** stores float values so number of bytes is 4 */
-  public static final int BYTES = Float.BYTES;
-
-  /**
-   * Create a new FloatRangeField type, from min/max parallel arrays
-   *
-   * @param name field name. must not be null.
-   * @param min range min values; each entry is the min value for the dimension
-   * @param max range max values; each entry is the max value for the dimension
-   */
-  public FloatRangeField(String name, final float[] min, final float[] max) {
-    super(name, getType(min.length));
-    setRangeValues(min, max);
-  }
-
-  /** set the field type */
-  private static FieldType getType(int dimensions) {
-    if (dimensions > 4) {
-      throw new IllegalArgumentException("FloatRangeField does not support greater than 4 dimensions");
-    }
-
-    FieldType ft = new FieldType();
-    // dimensions is set as 2*dimension size (min/max per dimension)
-    ft.setDimensions(dimensions*2, BYTES);
-    ft.freeze();
-    return ft;
-  }
-
-  /**
-   * Changes the values of the field.
-   * @param min array of min values. (accepts {@code Float.NEGATIVE_INFINITY})
-   * @param max array of max values. (accepts {@code Float.POSITIVE_INFINITY})
-   * @throws IllegalArgumentException if {@code min} or {@code max} is invalid
-   */
-  public void setRangeValues(float[] min, float[] max) {
-    checkArgs(min, max);
-    if (min.length*2 != type.pointDimensionCount() || max.length*2 != type.pointDimensionCount()) {
-      throw new IllegalArgumentException("field (name=" + name + ") uses " + type.pointDimensionCount()/2
-          + " dimensions; cannot change to (incoming) " + min.length + " dimensions");
-    }
-
-    final byte[] bytes;
-    if (fieldsData == null) {
-      bytes = new byte[BYTES*2*min.length];
-      fieldsData = new BytesRef(bytes);
-    } else {
-      bytes = ((BytesRef)fieldsData).bytes;
-    }
-    verifyAndEncode(min, max, bytes);
-  }
-
-  /** validate the arguments */
-  private static void checkArgs(final float[] min, final float[] max) {
-    if (min == null || max == null || min.length == 0 || max.length == 0) {
-      throw new IllegalArgumentException("min/max range values cannot be null or empty");
-    }
-    if (min.length != max.length) {
-      throw new IllegalArgumentException("min/max ranges must agree");
-    }
-    if (min.length > 4) {
-      throw new IllegalArgumentException("FloatRangeField does not support greater than 4 dimensions");
-    }
-  }
-
-  /**
-   * Encodes the min, max ranges into a byte array
-   */
-  private static byte[] encode(float[] min, float[] max) {
-    checkArgs(min, max);
-    byte[] b = new byte[BYTES*2*min.length];
-    verifyAndEncode(min, max, b);
-    return b;
-  }
-
-  /**
-   * encode the ranges into a sortable byte array ({@code Float.NaN} not allowed)
-   * <p>
-   * example for 4 dimensions (8 bytes per dimension value):
-   * minD1 ... minD4 | maxD1 ... maxD4
-   */
-  static void verifyAndEncode(float[] min, float[] max, byte[] bytes) {
-    for (int d=0,i=0,j=min.length*BYTES; d<min.length; ++d, i+=BYTES, j+=BYTES) {
-      if (Double.isNaN(min[d])) {
-        throw new IllegalArgumentException("invalid min value (" + Float.NaN + ")" + " in FloatRangeField");
-      }
-      if (Double.isNaN(max[d])) {
-        throw new IllegalArgumentException("invalid max value (" + Float.NaN + ")" + " in FloatRangeField");
-      }
-      if (min[d] > max[d]) {
-        throw new IllegalArgumentException("min value (" + min[d] + ") is greater than max value (" + max[d] + ")");
-      }
-      encode(min[d], bytes, i);
-      encode(max[d], bytes, j);
-    }
-  }
-
-  /** encode the given value into the byte array at the defined offset */
-  private static void encode(float val, byte[] bytes, int offset) {
-    NumericUtils.intToSortableBytes(NumericUtils.floatToSortableInt(val), bytes, offset);
-  }
-
-  /**
-   * Get the min value for the given dimension
-   * @param dimension the dimension, always positive
-   * @return the decoded min value
-   */
-  public float getMin(int dimension) {
-    if (dimension < 0 || dimension >= type.pointDimensionCount()/2) {
-      throw new IllegalArgumentException("dimension request (" + dimension +
-          ") out of bounds for field (name=" + name + " dimensions=" + type.pointDimensionCount()/2 + "). ");
-    }
-    return decodeMin(((BytesRef)fieldsData).bytes, dimension);
-  }
-
-  /**
-   * Get the max value for the given dimension
-   * @param dimension the dimension, always positive
-   * @return the decoded max value
-   */
-  public float getMax(int dimension) {
-    if (dimension < 0 || dimension >= type.pointDimensionCount()/2) {
-      throw new IllegalArgumentException("dimension request (" + dimension +
-          ") out of bounds for field (name=" + name + " dimensions=" + type.pointDimensionCount()/2 + "). ");
-    }
-    return decodeMax(((BytesRef)fieldsData).bytes, dimension);
-  }
-
-  /** decodes the min value (for the defined dimension) from the encoded input byte array */
-  static float decodeMin(byte[] b, int dimension) {
-    int offset = dimension*BYTES;
-    return NumericUtils.sortableIntToFloat(NumericUtils.sortableBytesToInt(b, offset));
-  }
-
-  /** decodes the max value (for the defined dimension) from the encoded input byte array */
-  static float decodeMax(byte[] b, int dimension) {
-    int offset = b.length/2 + dimension*BYTES;
-    return NumericUtils.sortableIntToFloat(NumericUtils.sortableBytesToInt(b, offset));
-  }
-
-  /**
-   * Create a query for matching indexed ranges that intersect the defined range.
-   * @param field field name. must not be null.
-   * @param min array of min values. (accepts {@code Float.NEGATIVE_INFINITY})
-   * @param max array of max values. (accepts {@code Float.MAX_VALUE})
-   * @return query for matching intersecting ranges (overlap, within, or contains)
-   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
-   */
-  public static Query newIntersectsQuery(String field, final float[] min, final float[] max) {
-    return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.INTERSECTS) {
-      @Override
-      protected String toString(byte[] ranges, int dimension) {
-        return FloatRangeField.toString(ranges, dimension);
-      }
-    };
-  }
-
-  /**
-   * Create a query for matching indexed float ranges that contain the defined range.
-   * @param field field name. must not be null.
-   * @param min array of min values. (accepts {@code Float.NEGATIVE_INFINITY})
-   * @param max array of max values. (accepts {@code Float.POSITIVE_INFINITY})
-   * @return query for matching ranges that contain the defined range
-   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
-   */
-  public static Query newContainsQuery(String field, final float[] min, final float[] max) {
-    return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.CONTAINS) {
-      @Override
-      protected String toString(byte[] ranges, int dimension) {
-        return FloatRangeField.toString(ranges, dimension);
-      }
-    };
-  }
-
-  /**
-   * Create a query for matching indexed ranges that are within the defined range.
-   * @param field field name. must not be null.
-   * @param min array of min values. (accepts {@code Float.NEGATIVE_INFINITY})
-   * @param max array of max values. (accepts {@code Float.POSITIVE_INFINITY})
-   * @return query for matching ranges within the defined range
-   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
-   */
-  public static Query newWithinQuery(String field, final float[] min, final float[] max) {
-    checkArgs(min, max);
-    return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.WITHIN) {
-      @Override
-      protected String toString(byte[] ranges, int dimension) {
-        return FloatRangeField.toString(ranges, dimension);
-      }
-    };
-  }
-
-  /**
-   * Create a query for matching indexed ranges that cross the defined range.
-   * A CROSSES is defined as any set of ranges that are not disjoint and not wholly contained by
-   * the query. Effectively, its the complement of union(WITHIN, DISJOINT).
-   * @param field field name. must not be null.
-   * @param min array of min values. (accepts {@code Float.NEGATIVE_INFINITY})
-   * @param max array of max values. (accepts {@code Float.POSITIVE_INFINITY})
-   * @return query for matching ranges within the defined range
-   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
-   */
-  public static Query newCrossesQuery(String field, final float[] min, final float[] max) {
-    checkArgs(min, max);
-    return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.CROSSES) {
-      @Override
-      protected String toString(byte[] ranges, int dimension) {
-        return FloatRangeField.toString(ranges, dimension);
-      }
-    };
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder();
-    sb.append(getClass().getSimpleName());
-    sb.append(" <");
-    sb.append(name);
-    sb.append(':');
-    byte[] b = ((BytesRef)fieldsData).bytes;
-    toString(b, 0);
-    for (int d=1; d<type.pointDimensionCount(); ++d) {
-      sb.append(' ');
-      toString(b, d);
-    }
-    sb.append('>');
-
-    return sb.toString();
-  }
-
-  /**
-   * Returns the String representation for the range at the given dimension
-   * @param ranges the encoded ranges, never null
-   * @param dimension the dimension of interest
-   * @return The string representation for the range at the provided dimension
-   */
-  private static String toString(byte[] ranges, int dimension) {
-    return "[" + Float.toString(decodeMin(ranges, dimension)) + " : "
-        + Float.toString(decodeMax(ranges, dimension)) + "]";
-  }
-}


[08/43] lucene-solr:feature/autoscaling: SOLR-10235: Fix DIH's TestJdbcDataSource to work with Java 9 and other Java runtimes that do not use the same DriverManager implementation like Oracle's original one

Posted by sh...@apache.org.
SOLR-10235: Fix DIH's TestJdbcDataSource to work with Java 9 and other Java runtimes that do not use the same DriverManager implementation like Oracle's original one


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

Branch: refs/heads/feature/autoscaling
Commit: 0d2c027857bfca3486399b0e6b19a5887081287a
Parents: 2d51a42
Author: Uwe Schindler <us...@apache.org>
Authored: Tue Mar 7 19:01:15 2017 +0100
Committer: Uwe Schindler <us...@apache.org>
Committed: Tue Mar 7 19:01:15 2017 +0100

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  6 ++
 .../handler/dataimport/TestJdbcDataSource.java  | 78 +++++++++++++++++---
 2 files changed, 73 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0d2c0278/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 5b0eb03..dc97456 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -300,6 +300,12 @@ Other Changes
 
 * SOLR-10230: default TTL of PKIAuthenticationPlugin increased to 10secs (noble)
 
+* SOLR-10235: Fix DIH's TestJdbcDataSource to work with Java 9 and other Java runtimes that
+  do not use the same DriverManager implementation like Oracle's original one. The test now
+  uses a fully implemented Driver instance returning a mock connection. The test also works
+  correct now if other drivers were installed before test execution (e.g., through IDE).
+  (hossman, Uwe Schindler)
+
 ==================  6.4.2 ==================
 
 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/0d2c0278/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestJdbcDataSource.java
----------------------------------------------------------------------
diff --git a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestJdbcDataSource.java b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestJdbcDataSource.java
index 2f6b24c..dcb4dbc 100644
--- a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestJdbcDataSource.java
+++ b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestJdbcDataSource.java
@@ -35,7 +35,6 @@ import java.util.Properties;
 
 import javax.sql.DataSource;
 
-import org.apache.lucene.util.Constants;
 import org.apache.solr.handler.dataimport.JdbcDataSource.ResultSetIterator;
 import static org.mockito.Mockito.*;
 import org.junit.After;
@@ -485,17 +484,16 @@ public class TestJdbcDataSource extends AbstractDataImportHandlerTestCase {
 
   @Test
   public void testRetrieveFromDriverManager() throws Exception {
-    assumeFalse("In Java 9, Class.forName() does not work for mock classes", Constants.JRE_IS_MINIMUM_JAVA9);
-    DriverManager.registerDriver(driver);
+    // we're not (directly) using a Mockito based mock class here because it won't have a consistent class name
+    // that will work with DriverManager's class bindings
+    MockDriver mockDriver = new MockDriver(connection);
+    DriverManager.registerDriver(mockDriver);
     try {
-      when(driver.connect(notNull(),notNull())).thenReturn(connection);
-
-      props.put(JdbcDataSource.DRIVER, driver.getClass().getName());
-      props.put(JdbcDataSource.URL, "jdbc:fakedb");
+      props.put(JdbcDataSource.DRIVER, MockDriver.class.getName());
+      props.put(JdbcDataSource.URL, MockDriver.MY_JDBC_URL);
       props.put("holdability", "HOLD_CURSORS_OVER_COMMIT");
 
-      Connection conn = jdbcDataSource.createConnectionFactory(context, props)
-              .call();
+      Connection conn = jdbcDataSource.createConnectionFactory(context, props).call();
 
       verify(connection).setAutoCommit(false);
       verify(connection).setHoldability(1);
@@ -504,7 +502,7 @@ public class TestJdbcDataSource extends AbstractDataImportHandlerTestCase {
     } catch(Exception e) {
       throw e;
     } finally {
-      DriverManager.deregisterDriver(driver);
+      DriverManager.deregisterDriver(mockDriver);
     }
   }
 
@@ -594,5 +592,63 @@ public class TestJdbcDataSource extends AbstractDataImportHandlerTestCase {
     byte[] content = "secret".getBytes(StandardCharsets.UTF_8);
     createFile(tmpdir, "enckeyfile.txt", content, false);
     return new File(tmpdir, "enckeyfile.txt").getAbsolutePath();
-  }  
+  }
+
+  /**
+   * A stub driver that returns our mocked connection for connection URL {@link #MY_JDBC_URL}.
+   * <p>
+   * This class is used instead of a Mockito mock because {@link DriverManager} uses the class
+   * name to lookup the driver and also requires the driver to behave in a sane way, if other
+   * drivers are registered in the runtime. A simple Mockito mock is likely to break
+   * depending on JVM runtime version. So this class implements a full {@link Driver},
+   * so {@code DriverManager} can do whatever it wants to find the correct driver for a URL.
+   */
+  public static final class MockDriver implements Driver {
+    public static final String MY_JDBC_URL = "jdbc:fakedb";
+    private final Connection conn;
+    
+    public MockDriver() throws SQLException {
+      throw new AssertionError("The driver should never be directly instantiated by DIH's JdbcDataSource");
+    }
+    
+    MockDriver(Connection conn) throws SQLException {
+      this.conn = conn;
+    }
+    
+    @Override
+    public boolean acceptsURL(String url) throws java.sql.SQLException {
+      return MY_JDBC_URL.equals(url);
+    }
+    
+    @Override
+    public Connection connect(String url, Properties info) throws java.sql.SQLException {
+      return acceptsURL(url) ? conn : null;
+    }
+    
+    @Override
+    public int getMajorVersion() {
+      return 1;
+    }
+    
+    @Override
+    public int getMinorVersion() {
+      return 0;
+    }
+    
+    @Override
+    public java.util.logging.Logger getParentLogger() throws java.sql.SQLFeatureNotSupportedException {
+      throw new java.sql.SQLFeatureNotSupportedException();
+    }
+    
+    @Override
+    public java.sql.DriverPropertyInfo[] getPropertyInfo(String url, Properties info) throws SQLException {
+      return new java.sql.DriverPropertyInfo[0];
+    }
+    
+    @Override
+    public boolean jdbcCompliant() {
+      // we are not fully compliant:
+      return false;
+    }
+  }
 }


[22/43] lucene-solr:feature/autoscaling: SOLR-10257: Add logarithm StreamEvaluator

Posted by sh...@apache.org.
SOLR-10257: Add logarithm StreamEvaluator


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

Branch: refs/heads/feature/autoscaling
Commit: d945a246f6071699790119f07a66fb4c5505cee2
Parents: c680f45
Author: Joel Bernstein <jb...@apache.org>
Authored: Thu Mar 9 20:48:14 2017 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Thu Mar 9 21:01:24 2017 -0500

----------------------------------------------------------------------
 .../org/apache/solr/handler/StreamHandler.java  |  3 +-
 .../solrj/io/eval/NaturalLogEvaluator.java      | 60 ++++++++++++
 .../io/stream/eval/NaturalLogEvaluatorTest.java | 98 ++++++++++++++++++++
 3 files changed, 160 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d945a246/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
index 06e59b6..e69f52b 100644
--- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
@@ -42,6 +42,7 @@ import org.apache.solr.client.solrj.io.eval.IfThenElseEvaluator;
 import org.apache.solr.client.solrj.io.eval.LessThanEqualToEvaluator;
 import org.apache.solr.client.solrj.io.eval.LessThanEvaluator;
 import org.apache.solr.client.solrj.io.eval.MultiplyEvaluator;
+import org.apache.solr.client.solrj.io.eval.NaturalLogEvaluator;
 import org.apache.solr.client.solrj.io.eval.NotEvaluator;
 import org.apache.solr.client.solrj.io.eval.OrEvaluator;
 import org.apache.solr.client.solrj.io.eval.RawValueEvaluator;
@@ -197,7 +198,7 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
       .withFunctionName("div", DivideEvaluator.class)
       .withFunctionName("mult", MultiplyEvaluator.class)
       .withFunctionName("sub", SubtractEvaluator.class)
-      
+      .withFunctionName("log", NaturalLogEvaluator.class)
       // Conditional Stream Evaluators
       .withFunctionName("if", IfThenElseEvaluator.class)
       ;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d945a246/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/NaturalLogEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/NaturalLogEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/NaturalLogEvaluator.java
new file mode 100644
index 0000000..19709e6
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/NaturalLogEvaluator.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.client.solrj.io.eval;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class NaturalLogEvaluator extends NumberEvaluator {
+  protected static final long serialVersionUID = 1L;
+
+  public NaturalLogEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
+    super(expression, factory);
+
+    if(1 != subEvaluators.size()){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting one value but found %d",expression,subEvaluators.size()));
+    }
+  }
+
+  @Override
+  public Number evaluate(Tuple tuple) throws IOException {
+
+    List<BigDecimal> results = evaluateAll(tuple);
+
+    // we're still doing these checks because if we ever add an array-flatten evaluator,
+    // one found in the constructor could become != 1
+    if(1 != results.size()){
+      throw new IOException(String.format(Locale.ROOT,"%s(...) only works with a 1 value but %d were provided", constructingFactory.getFunctionName(getClass()), results.size()));
+    }
+
+    if(null == results.get(0)){
+      return null;
+    }
+
+    return Math.log(results.get(0).doubleValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d945a246/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/NaturalLogEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/NaturalLogEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/NaturalLogEvaluatorTest.java
new file mode 100644
index 0000000..c4ae127
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/NaturalLogEvaluatorTest.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.solr.client.solrj.io.stream.eval;
+
+import java.io.IOException;
+import java.util.Map;
+
+import junit.framework.Assert;
+
+import org.apache.commons.collections.map.HashedMap;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.eval.AddEvaluator;
+import org.apache.solr.client.solrj.io.eval.NaturalLogEvaluator;
+import org.apache.solr.client.solrj.io.eval.StreamEvaluator;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+import org.junit.Test;
+
+public class NaturalLogEvaluatorTest extends LuceneTestCase {
+
+  StreamFactory factory;
+  Map<String, Object> values;
+
+  public NaturalLogEvaluatorTest() {
+    super();
+
+    factory = new StreamFactory()
+        .withFunctionName("log", NaturalLogEvaluator.class).withFunctionName("add", AddEvaluator.class);
+    values = new HashedMap();
+  }
+
+  @Test
+  public void logOneField() throws Exception{
+    StreamEvaluator evaluator = factory.constructEvaluator("log(a)");
+    Object result;
+
+    values.clear();
+    values.put("a", 100);
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertTrue(result instanceof Double);
+    Assert.assertTrue(result.equals(Math.log(100)));
+
+  }
+
+  @Test
+  public void logNestedField() throws Exception{
+    StreamEvaluator evaluator = factory.constructEvaluator("log(add(50,50))");
+    Object result;
+
+    values.clear();
+    result = evaluator.evaluate(new Tuple(values));
+    Assert.assertTrue(result instanceof Double);
+    Assert.assertTrue(result.equals(Math.log(100)));
+
+  }
+
+  @Test(expected = IOException.class)
+  public void logNoField() throws Exception{
+    factory.constructEvaluator("log()");
+  }
+
+  @Test(expected = IOException.class)
+  public void logTwoFields() throws Exception{
+    factory.constructEvaluator("log(a,b)");
+  }
+
+  @Test
+  public void logNoValue() throws Exception{
+    StreamEvaluator evaluator = factory.constructEvaluator("log(a)");
+
+    values.clear();
+    Object result = evaluator.evaluate(new Tuple(values));
+    assertNull(result);
+  }
+  @Test
+  public void logNullValue() throws Exception{
+    StreamEvaluator evaluator = factory.constructEvaluator("log(a)");
+
+    values.clear();
+    values.put("a", null);
+    Object result = evaluator.evaluate(new Tuple(values));
+    assertNull(result);
+  }
+}


[12/43] lucene-solr:feature/autoscaling: SOLR-9858: Collect aggregated metrics from nodes and shard leaders in overseer.

Posted by sh...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d7bc947/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrReporter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrReporter.java b/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrReporter.java
new file mode 100644
index 0000000..e9b8c3d
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrReporter.java
@@ -0,0 +1,392 @@
+/*
+ * 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.solr;
+
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.regex.PatternSyntaxException;
+
+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.ScheduledReporter;
+import com.codahale.metrics.Timer;
+import org.apache.http.client.HttpClient;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.io.SolrClientCache;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.handler.admin.MetricsCollectorHandler;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.util.stats.MetricUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implementation of {@link ScheduledReporter} that reports metrics from selected registries and sends
+ * them periodically as update requests to a selected Solr collection and to a configured handler.
+ */
+public class SolrReporter extends ScheduledReporter {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public static final String REGISTRY_ID = "_registry_";
+  public static final String REPORTER_ID = "_reporter_";
+  public static final String GROUP_ID = "_group_";
+  public static final String LABEL_ID = "_label_";
+
+
+  /**
+   * Specification of what registries and what metrics to send.
+   */
+  public static final class Report {
+    public String groupPattern;
+    public String labelPattern;
+    public String registryPattern;
+    public Set<String> metricFilters = new HashSet<>();
+
+    /**
+     * Create a report specification
+     * @param groupPattern logical group for these metrics. This is used in {@link MetricsCollectorHandler}
+     *                     to select the target registry for metrics to aggregate. Must not be null or empty.
+     *                     It may contain back-references to capture groups from {@code registryPattern}
+     * @param labelPattern name of this group of metrics. This is used in {@link MetricsCollectorHandler}
+     *                     to prefix metric names. May be null or empty. It may contain back-references
+     *                     to capture groups from {@code registryPattern}.
+     * @param registryPattern pattern for selecting matching registries, see {@link SolrMetricManager#registryNames(String...)}
+     * @param metricFilters patterns for selecting matching metrics, see {@link org.apache.solr.metrics.SolrMetricManager.RegexFilter}
+     */
+    public Report(String groupPattern, String labelPattern, String registryPattern, Collection<String> metricFilters) {
+      this.groupPattern = groupPattern;
+      this.labelPattern = labelPattern;
+      this.registryPattern = registryPattern;
+      if (metricFilters != null) {
+        this.metricFilters.addAll(metricFilters);
+      }
+    }
+
+    public static Report fromMap(Map<?, ?> map) {
+      String groupPattern = (String)map.get("group");
+      String labelPattern = (String)map.get("label");
+      String registryPattern = (String)map.get("registry");
+      Object oFilters = map.get("filter");
+      Collection<String> metricFilters = Collections.emptyList();
+      if (oFilters != null) {
+        if (oFilters instanceof String) {
+          metricFilters = Collections.singletonList((String)oFilters);
+        } else if (oFilters instanceof Collection) {
+          metricFilters = (Collection<String>)oFilters;
+        } else {
+          log.warn("Invalid report filters, ignoring: " + oFilters);
+        }
+      }
+      if (groupPattern == null || registryPattern == null) {
+        log.warn("Invalid report configuration, group and registry required!: " + map);
+        return null;
+      }
+      return new Report(groupPattern, labelPattern, registryPattern, metricFilters);
+    }
+  }
+
+  public static class Builder {
+    private final SolrMetricManager metricManager;
+    private final List<Report> reports;
+    private String reporterId;
+    private TimeUnit rateUnit;
+    private TimeUnit durationUnit;
+    private String handler;
+    private boolean skipHistograms;
+    private boolean skipAggregateValues;
+    private boolean cloudClient;
+    private SolrParams params;
+
+    /**
+     * Create a builder for SolrReporter.
+     * @param metricManager metric manager that is the source of metrics
+     * @param reports report definitions
+     * @return builder
+     */
+    public static Builder forReports(SolrMetricManager metricManager, List<Report> reports) {
+      return new Builder(metricManager, reports);
+    }
+
+    private Builder(SolrMetricManager metricManager, List<Report> reports) {
+      this.metricManager = metricManager;
+      this.reports = reports;
+      this.rateUnit = TimeUnit.SECONDS;
+      this.durationUnit = TimeUnit.MILLISECONDS;
+      this.skipHistograms = false;
+      this.skipAggregateValues = false;
+      this.cloudClient = false;
+      this.params = null;
+    }
+
+    /**
+     * Additional {@link SolrParams} to add to every request.
+     * @param params additional params
+     * @return {@code this}
+     */
+    public Builder withSolrParams(SolrParams params) {
+      this.params = params;
+      return this;
+    }
+    /**
+     * If true then use {@link org.apache.solr.client.solrj.impl.CloudSolrClient} for communication.
+     * Default is false.
+     * @param cloudClient use CloudSolrClient when true, {@link org.apache.solr.client.solrj.impl.HttpSolrClient} otherwise.
+     * @return {@code this}
+     */
+    public Builder cloudClient(boolean cloudClient) {
+      this.cloudClient = cloudClient;
+      return this;
+    }
+
+    /**
+     * Histograms are difficult / impossible to aggregate, so it may not be
+     * worth to report them.
+     * @param skipHistograms when true then skip histograms from reports
+     * @return {@code this}
+     */
+    public Builder skipHistograms(boolean skipHistograms) {
+      this.skipHistograms = skipHistograms;
+      return this;
+    }
+
+    /**
+     * Individual values from {@link org.apache.solr.metrics.AggregateMetric} may not be worth to report.
+     * @param skipAggregateValues when tru then skip reporting individual values from the metric
+     * @return {@code this}
+     */
+    public Builder skipAggregateValues(boolean skipAggregateValues) {
+      this.skipAggregateValues = skipAggregateValues;
+      return this;
+    }
+
+    /**
+     * Handler name to use at the remote end.
+     *
+     * @param handler handler name, eg. "/admin/metricsCollector"
+     * @return {@code this}
+     */
+    public Builder withHandler(String handler) {
+      this.handler = handler;
+      return this;
+    }
+
+    /**
+     * Use this id to identify metrics from this instance.
+     *
+     * @param reporterId reporter id
+     * @return {@code this}
+     */
+    public Builder withReporterId(String reporterId) {
+      this.reporterId = reporterId;
+      return this;
+    }
+
+    /**
+     * Convert rates to the given time unit.
+     *
+     * @param rateUnit a unit of time
+     * @return {@code this}
+     */
+    public Builder convertRatesTo(TimeUnit rateUnit) {
+      this.rateUnit = rateUnit;
+      return this;
+    }
+
+    /**
+     * Convert durations to the given time unit.
+     *
+     * @param durationUnit a unit of time
+     * @return {@code this}
+     */
+    public Builder convertDurationsTo(TimeUnit durationUnit) {
+      this.durationUnit = durationUnit;
+      return this;
+    }
+
+    /**
+     * Build it.
+     * @param client an instance of {@link HttpClient} to be used for making calls.
+     * @param urlProvider function that returns the base URL of Solr instance to target. May return
+     *                    null to indicate that reporting should be skipped. Note: this
+     *                    function will be called every time just before report is sent.
+     * @return configured instance of reporter
+     */
+    public SolrReporter build(HttpClient client, Supplier<String> urlProvider) {
+      return new SolrReporter(client, urlProvider, metricManager, reports, handler, reporterId, rateUnit, durationUnit,
+          params, skipHistograms, skipAggregateValues, cloudClient);
+    }
+
+  }
+
+  private String reporterId;
+  private String handler;
+  private Supplier<String> urlProvider;
+  private SolrClientCache clientCache;
+  private List<CompiledReport> compiledReports;
+  private SolrMetricManager metricManager;
+  private boolean skipHistograms;
+  private boolean skipAggregateValues;
+  private boolean cloudClient;
+  private ModifiableSolrParams params;
+  private Map<String, Object> metadata;
+
+  private static final class CompiledReport {
+    String group;
+    String label;
+    Pattern registryPattern;
+    MetricFilter filter;
+
+    CompiledReport(Report report) throws PatternSyntaxException {
+      this.group = report.groupPattern;
+      this.label = report.labelPattern;
+      this.registryPattern = Pattern.compile(report.registryPattern);
+      this.filter = new SolrMetricManager.RegexFilter(report.metricFilters);
+    }
+
+    @Override
+    public String toString() {
+      return "CompiledReport{" +
+          "group='" + group + '\'' +
+          ", label='" + label + '\'' +
+          ", registryPattern=" + registryPattern +
+          ", filter=" + filter +
+          '}';
+    }
+  }
+
+  public SolrReporter(HttpClient httpClient, Supplier<String> urlProvider, SolrMetricManager metricManager,
+                      List<Report> metrics, String handler,
+                      String reporterId, TimeUnit rateUnit, TimeUnit durationUnit,
+                      SolrParams params, boolean skipHistograms, boolean skipAggregateValues, boolean cloudClient) {
+    super(null, "solr-reporter", MetricFilter.ALL, rateUnit, durationUnit);
+    this.metricManager = metricManager;
+    this.urlProvider = urlProvider;
+    this.reporterId = reporterId;
+    if (handler == null) {
+      handler = MetricsCollectorHandler.HANDLER_PATH;
+    }
+    this.handler = handler;
+    this.clientCache = new SolrClientCache(httpClient);
+    this.compiledReports = new ArrayList<>();
+    metrics.forEach(report -> {
+      MetricFilter filter = new SolrMetricManager.RegexFilter(report.metricFilters);
+      try {
+        CompiledReport cs = new CompiledReport(report);
+        compiledReports.add(cs);
+      } catch (PatternSyntaxException e) {
+        log.warn("Skipping report with invalid registryPattern: " + report.registryPattern, e);
+      }
+    });
+    this.skipHistograms = skipHistograms;
+    this.skipAggregateValues = skipAggregateValues;
+    this.cloudClient = cloudClient;
+    this.params = new ModifiableSolrParams();
+    this.params.set(REPORTER_ID, reporterId);
+    // allow overrides to take precedence
+    if (params != null) {
+      this.params.add(params);
+    }
+    metadata = new HashMap<>();
+    metadata.put(REPORTER_ID, reporterId);
+  }
+
+  @Override
+  public void close() {
+    clientCache.close();
+    super.close();
+  }
+
+  @Override
+  public void report() {
+    String url = urlProvider.get();
+    // if null then suppress reporting
+    if (url == null) {
+      return;
+    }
+
+    SolrClient solr;
+    if (cloudClient) {
+      solr = clientCache.getCloudSolrClient(url);
+    } else {
+      solr = clientCache.getHttpSolrClient(url);
+    }
+    UpdateRequest req = new UpdateRequest(handler);
+    req.setParams(params);
+    compiledReports.forEach(report -> {
+      Set<String> registryNames = metricManager.registryNames(report.registryPattern);
+      registryNames.forEach(registryName -> {
+        String label = report.label;
+        if (label != null && label.indexOf('$') != -1) {
+          // label with back-references
+          Matcher m = report.registryPattern.matcher(registryName);
+          label = m.replaceFirst(label);
+        }
+        final String effectiveLabel = label;
+        String group = report.group;
+        if (group.indexOf('$') != -1) {
+          // group with back-references
+          Matcher m = report.registryPattern.matcher(registryName);
+          group = m.replaceFirst(group);
+        }
+        final String effectiveGroup = group;
+        MetricUtils.toSolrInputDocuments(metricManager.registry(registryName), Collections.singletonList(report.filter), MetricFilter.ALL,
+            skipHistograms, skipAggregateValues, metadata, doc -> {
+              doc.setField(REGISTRY_ID, registryName);
+              doc.setField(GROUP_ID, effectiveGroup);
+              if (effectiveLabel != null) {
+                doc.setField(LABEL_ID, effectiveLabel);
+              }
+              req.add(doc);
+            });
+      });
+    });
+
+    // if no docs added then don't send a report
+    if (req.getDocuments() == null || req.getDocuments().isEmpty()) {
+      return;
+    }
+    try {
+      //log.info("%%% sending to " + url + ": " + req.getParams());
+      solr.request(req);
+    } catch (Exception e) {
+      log.debug("Error sending metric report", e.toString());
+    }
+
+  }
+
+  @Override
+  public void report(SortedMap<String, Gauge> gauges, SortedMap<String, Counter> counters, SortedMap<String, Histogram> histograms, SortedMap<String, Meter> meters, SortedMap<String, Timer> timers) {
+    // no-op - we do all the work in report()
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d7bc947/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrShardReporter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrShardReporter.java b/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrShardReporter.java
new file mode 100644
index 0000000..2b20274
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrShardReporter.java
@@ -0,0 +1,188 @@
+/*
+ * 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.solr;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+import org.apache.solr.cloud.CloudDescriptor;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.handler.admin.MetricsCollectorHandler;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.metrics.SolrMetricReporter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class reports selected metrics from replicas to shard leader.
+ * <p>The following configuration properties are supported:</p>
+ * <ul>
+ *   <li>handler - (optional str) handler path where reports are sent. Default is
+ *   {@link MetricsCollectorHandler#HANDLER_PATH}.</li>
+ *   <li>period - (optional int) how often reports are sent, in seconds. Default is 60. Setting this
+ *   to 0 disables the reporter.</li>
+ *   <li>filter - (optional multiple str) regex expression(s) matching selected metrics to be reported.</li>
+ * </ul>
+ * NOTE: this reporter uses predefined "replica" group, and it's always created even if explicit configuration
+ * is missing. Default configuration uses filters defined in {@link #DEFAULT_FILTERS}.
+ * <p>Example configuration:</p>
+ * <pre>
+ *    &lt;reporter name="test" group="replica"&gt;
+ *      &lt;int name="period"&gt;11&lt;/int&gt;
+ *      &lt;str name="filter"&gt;UPDATE\./update/.*requests&lt;/str&gt;
+ *      &lt;str name="filter"&gt;QUERY\./select.*requests&lt;/str&gt;
+ *    &lt;/reporter&gt;
+ * </pre>
+ */
+public class SolrShardReporter extends SolrMetricReporter {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public static final List<String> DEFAULT_FILTERS = new ArrayList(){{
+    add("TLOG.*");
+    add("REPLICATION.*");
+    add("INDEX.flush.*");
+    add("INDEX.merge.major.*");
+    add("UPDATE\\./update/.*requests");
+    add("QUERY\\./select.*requests");
+  }};
+
+  private String handler = MetricsCollectorHandler.HANDLER_PATH;
+  private int period = SolrMetricManager.DEFAULT_CLOUD_REPORTER_PERIOD;
+  private List<String> filters = new ArrayList<>();
+
+  private SolrReporter reporter;
+
+  /**
+   * Create a reporter for metrics managed in a named registry.
+   *
+   * @param metricManager metric manager
+   * @param registryName  registry to use, one of registries managed by
+   *                      {@link SolrMetricManager}
+   */
+  public SolrShardReporter(SolrMetricManager metricManager, String registryName) {
+    super(metricManager, registryName);
+  }
+
+  public void setHandler(String handler) {
+    this.handler = handler;
+  }
+
+  public void setPeriod(int period) {
+    this.period = period;
+  }
+
+  public void setFilter(List<String> filterConfig) {
+    if (filterConfig == null || filterConfig.isEmpty()) {
+      return;
+    }
+    filters = filterConfig;
+  }
+
+  // for unit tests
+  int getPeriod() {
+    return period;
+  }
+
+  @Override
+  protected void validate() throws IllegalStateException {
+    if (period < 1) {
+      log.info("Turning off shard reporter, period=" + period);
+    }
+    if (filters.isEmpty()) {
+      filters = DEFAULT_FILTERS;
+    }
+    // start in inform(...) only when core is available
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (reporter != null) {
+      reporter.close();
+    }
+  }
+
+  public void setCore(SolrCore core) {
+    if (reporter != null) {
+      reporter.close();
+    }
+    if (core.getCoreDescriptor().getCloudDescriptor() == null) {
+      // not a cloud core
+      log.warn("Not initializing shard reporter for non-cloud core " + core.getName());
+      return;
+    }
+    if (period < 1) { // don't start it
+      log.warn("Not starting shard reporter ");
+      return;
+    }
+    // our id is coreNodeName
+    String id = core.getCoreDescriptor().getCloudDescriptor().getCoreNodeName();
+    // target registry is the leaderRegistryName
+    String groupId = core.getCoreMetricManager().getLeaderRegistryName();
+    if (groupId == null) {
+      log.warn("No leaderRegistryName for core " + core + ", not starting the reporter...");
+      return;
+    }
+    SolrReporter.Report spec = new SolrReporter.Report(groupId, null, registryName, filters);
+    reporter = SolrReporter.Builder.forReports(metricManager, Collections.singletonList(spec))
+        .convertRatesTo(TimeUnit.SECONDS)
+        .convertDurationsTo(TimeUnit.MILLISECONDS)
+        .withHandler(handler)
+        .withReporterId(id)
+        .cloudClient(false) // we want to send reports specifically to a selected leader instance
+        .skipAggregateValues(true) // we don't want to transport details of aggregates
+        .skipHistograms(true) // we don't want to transport histograms
+        .build(core.getCoreDescriptor().getCoreContainer().getUpdateShardHandler().getHttpClient(), new LeaderUrlSupplier(core));
+
+    reporter.start(period, TimeUnit.SECONDS);
+  }
+
+  private static class LeaderUrlSupplier implements Supplier<String> {
+    private SolrCore core;
+
+    LeaderUrlSupplier(SolrCore core) {
+      this.core = core;
+    }
+
+    @Override
+    public String get() {
+      CloudDescriptor cd = core.getCoreDescriptor().getCloudDescriptor();
+      if (cd == null) {
+        return null;
+      }
+      ClusterState state = core.getCoreDescriptor().getCoreContainer().getZkController().getClusterState();
+      DocCollection collection = state.getCollection(core.getCoreDescriptor().getCollectionName());
+      Replica replica = collection.getLeader(core.getCoreDescriptor().getCloudDescriptor().getShardId());
+      if (replica == null) {
+        log.warn("No leader for " + collection.getName() + "/" + core.getCoreDescriptor().getCloudDescriptor().getShardId());
+        return null;
+      }
+      String baseUrl = replica.getStr("base_url");
+      if (baseUrl == null) {
+        log.warn("No base_url for replica " + replica);
+      }
+      return baseUrl;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d7bc947/solr/core/src/java/org/apache/solr/metrics/reporters/solr/package-info.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/reporters/solr/package-info.java b/solr/core/src/java/org/apache/solr/metrics/reporters/solr/package-info.java
new file mode 100644
index 0000000..740bcce
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/metrics/reporters/solr/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 contains {@link org.apache.solr.metrics.SolrMetricReporter} implementations
+ * specific to SolrCloud reporting.
+ */
+package org.apache.solr.metrics.reporters.solr;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d7bc947/solr/core/src/java/org/apache/solr/update/PeerSync.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/PeerSync.java b/solr/core/src/java/org/apache/solr/update/PeerSync.java
index ac07413..874e39c 100644
--- a/solr/core/src/java/org/apache/solr/update/PeerSync.java
+++ b/solr/core/src/java/org/apache/solr/update/PeerSync.java
@@ -161,11 +161,13 @@ public class PeerSync implements SolrMetricProducer {
     core.getCoreMetricManager().registerMetricProducer(SolrInfoMBean.Category.REPLICATION.toString(), this);
   }
 
+  public static final String METRIC_SCOPE = "peerSync";
+
   @Override
   public void initializeMetrics(SolrMetricManager manager, String registry, String scope) {
-    syncTime = manager.timer(registry, "time", scope);
-    syncErrors = manager.counter(registry, "errors", scope);
-    syncSkipped = manager.counter(registry, "skipped", scope);
+    syncTime = manager.timer(registry, "time", scope, METRIC_SCOPE);
+    syncErrors = manager.counter(registry, "errors", scope, METRIC_SCOPE);
+    syncSkipped = manager.counter(registry, "skipped", scope, METRIC_SCOPE);
   }
 
   /** optional list of updates we had before possibly receiving new updates */

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d7bc947/solr/core/src/test-files/solr/solr-solrreporter.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/solr-solrreporter.xml b/solr/core/src/test-files/solr/solr-solrreporter.xml
new file mode 100644
index 0000000..db03e42
--- /dev/null
+++ b/solr/core/src/test-files/solr/solr-solrreporter.xml
@@ -0,0 +1,66 @@
+<?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>
+  <shardHandlerFactory name="shardHandlerFactory" class="HttpShardHandlerFactory">
+    <str name="urlScheme">${urlScheme:}</str>
+    <int name="socketTimeout">${socketTimeout:90000}</int>
+    <int name="connTimeout">${connTimeout:15000}</int>
+  </shardHandlerFactory>
+
+  <solrcloud>
+    <str name="host">127.0.0.1</str>
+    <int name="hostPort">${hostPort:8983}</int>
+    <str name="hostContext">${hostContext:solr}</str>
+    <int name="zkClientTimeout">${solr.zkclienttimeout:30000}</int>
+    <bool name="genericCoreNodeNames">${genericCoreNodeNames:true}</bool>
+    <int name="leaderVoteWait">${leaderVoteWait:10000}</int>
+    <int name="distribUpdateConnTimeout">${distribUpdateConnTimeout:45000}</int>
+    <int name="distribUpdateSoTimeout">${distribUpdateSoTimeout:340000}</int>
+    <int name="autoReplicaFailoverWaitAfterExpiration">${autoReplicaFailoverWaitAfterExpiration:10000}</int>
+    <int name="autoReplicaFailoverWorkLoopDelay">${autoReplicaFailoverWorkLoopDelay:10000}</int>
+    <int name="autoReplicaFailoverBadNodeExpiration">${autoReplicaFailoverBadNodeExpiration:60000}</int>
+  </solrcloud>
+
+  <metrics>
+    <reporter name="test" group="shard">
+      <int name="period">5</int>
+      <str name="filter">UPDATE\./update/.*requests</str>
+      <str name="filter">QUERY\./select.*requests</str>
+    </reporter>
+    <reporter name="test" group="cluster">
+      <str name="handler">/admin/metrics/collector</str>
+      <int name="period">5</int>
+      <lst name="report">
+        <str name="group">cluster</str>
+        <str name="label">jvm</str>
+        <str name="registry">solr\.jvm</str>
+        <str name="filter">memory\.total\..*</str>
+        <str name="filter">memory\.heap\..*</str>
+        <str name="filter">os\.SystemLoadAverage</str>
+        <str name="filter">threads\.count</str>
+      </lst>
+      <lst name="report">
+        <str name="group">cluster</str>
+        <str name="label">leader.$1</str>
+        <str name="registry">solr\.collection\.(.*)\.leader</str>
+        <str name="filter">UPDATE\./update/.*</str>
+      </lst>
+    </reporter>
+  </metrics>
+</solr>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d7bc947/solr/core/src/test/org/apache/solr/cloud/TestCloudRecovery.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestCloudRecovery.java b/solr/core/src/test/org/apache/solr/cloud/TestCloudRecovery.java
index 164eeab..1af09f4 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestCloudRecovery.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestCloudRecovery.java
@@ -119,9 +119,9 @@ public class TestCloudRecovery extends SolrCloudTestCase {
           .filter(s -> s.startsWith("solr.core.")).collect(Collectors.toList());
       for (String registry : registryNames) {
         Map<String, Metric> metrics = manager.registry(registry).getMetrics();
-        Timer timer = (Timer)metrics.get("REPLICATION.time");
-        Counter counter = (Counter)metrics.get("REPLICATION.errors");
-        Counter skipped = (Counter)metrics.get("REPLICATION.skipped");
+        Timer timer = (Timer)metrics.get("REPLICATION.peerSync.time");
+        Counter counter = (Counter)metrics.get("REPLICATION.peerSync.errors");
+        Counter skipped = (Counter)metrics.get("REPLICATION.peerSync.skipped");
         replicationCount += timer.getCount();
         errorsCount += counter.getCount();
         skippedCount += skipped.getCount();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d7bc947/solr/core/src/test/org/apache/solr/core/TestJmxMonitoredMap.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/TestJmxMonitoredMap.java b/solr/core/src/test/org/apache/solr/core/TestJmxMonitoredMap.java
index 2cad6e8..aa107bc 100644
--- a/solr/core/src/test/org/apache/solr/core/TestJmxMonitoredMap.java
+++ b/solr/core/src/test/org/apache/solr/core/TestJmxMonitoredMap.java
@@ -85,7 +85,7 @@ public class TestJmxMonitoredMap extends LuceneTestCase {
       log.info("Using port: " + port);
       String url = "service:jmx:rmi:///jndi/rmi://127.0.0.1:"+port+"/solrjmx";
       JmxConfiguration config = new JmxConfiguration(true, null, url, null);
-      monitoredMap = new JmxMonitoredMap<>("", "", config);
+      monitoredMap = new JmxMonitoredMap<>("", "", "", config);
       JMXServiceURL u = new JMXServiceURL(url);
       connector = JMXConnectorFactory.connect(u);
       mbeanServer = connector.getMBeanServerConnection();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d7bc947/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 1df6021..6e8e1e5 100644
--- a/solr/core/src/test/org/apache/solr/metrics/SolrCoreMetricManagerTest.java
+++ b/solr/core/src/test/org/apache/solr/metrics/SolrCoreMetricManagerTest.java
@@ -103,6 +103,7 @@ public class SolrCoreMetricManagerTest extends SolrTestCaseJ4 {
 
     String className = MockMetricReporter.class.getName();
     String reporterName = TestUtil.randomUnicodeString(random);
+    String taggedName = reporterName + "@" + coreMetricManager.getTag();
 
     Map<String, Object> attrs = new HashMap<>();
     attrs.put(FieldType.CLASS_NAME, className);
@@ -116,15 +117,16 @@ public class SolrCoreMetricManagerTest extends SolrTestCaseJ4 {
     PluginInfo pluginInfo = shouldDefinePlugin ? new PluginInfo(TestUtil.randomUnicodeString(random), attrs) : null;
 
     try {
-      metricManager.loadReporter(coreMetricManager.getRegistryName(), coreMetricManager.getCore().getResourceLoader(), pluginInfo);
+      metricManager.loadReporter(coreMetricManager.getRegistryName(), coreMetricManager.getCore().getResourceLoader(),
+          pluginInfo, String.valueOf(coreMetricManager.getCore().hashCode()));
       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);
+      assertNotNull("reporter " + reporterName + " not present among " + reporters, reporters.get(taggedName));
+      assertTrue("wrong reporter class: " + reporters.get(taggedName), reporters.get(taggedName) instanceof MockMetricReporter);
     } catch (IllegalArgumentException e) {
       assertTrue(pluginInfo == null || attrs.get("configurable") == null);
-      assertNull(metricManager.getReporters(coreMetricManager.getRegistryName()).get(reporterName));
+      assertNull(metricManager.getReporters(coreMetricManager.getRegistryName()).get(taggedName));
     }
   }
 
@@ -152,20 +154,11 @@ public class SolrCoreMetricManagerTest extends SolrTestCaseJ4 {
   }
 
   @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));
-
+  public void testNonCloudRegistryName() throws Exception {
+    String registryName = h.getCore().getCoreMetricManager().getRegistryName();
+    String leaderRegistryName = h.getCore().getCoreMetricManager().getLeaderRegistryName();
+    assertNotNull(registryName);
+    assertEquals("solr.core.collection1", registryName);
+    assertNull(leaderRegistryName);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d7bc947/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 ee2acd3..1c29c5e 100644
--- a/solr/core/src/test/org/apache/solr/metrics/SolrMetricManagerTest.java
+++ b/solr/core/src/test/org/apache/solr/metrics/SolrMetricManagerTest.java
@@ -205,32 +205,32 @@ public class SolrMetricManagerTest extends SolrTestCaseJ4 {
         createPluginInfo("node_foo", "node", null),
         createPluginInfo("core_foo", "core", null)
     };
-
-    metricManager.loadReporters(plugins, loader, SolrInfoMBean.Group.node);
+    String tag = "xyz";
+    metricManager.loadReporters(plugins, loader, tag, 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"));
+    assertTrue(reporters.containsKey("universal_foo@" + tag));
+    assertTrue(reporters.containsKey("multigroup_foo@" + tag));
+    assertTrue(reporters.containsKey("node_foo@" + tag));
+    assertTrue(reporters.containsKey("multiregistry_foo@" + tag));
 
-    metricManager.loadReporters(plugins, loader, SolrInfoMBean.Group.core, "collection1");
+    metricManager.loadReporters(plugins, loader, tag, 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"));
+    assertTrue(reporters.containsKey("universal_foo@" + tag));
+    assertTrue(reporters.containsKey("multigroup_foo@" + tag));
+    assertTrue(reporters.containsKey("specific_foo@" + tag));
+    assertTrue(reporters.containsKey("core_foo@" + tag));
+    assertTrue(reporters.containsKey("multiregistry_foo@" + tag));
 
-    metricManager.loadReporters(plugins, loader, SolrInfoMBean.Group.jvm);
+    metricManager.loadReporters(plugins, loader, tag, 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"));
+    assertTrue(reporters.containsKey("universal_foo@" + tag));
+    assertTrue(reporters.containsKey("multigroup_foo@" + tag));
 
     metricManager.removeRegistry("solr.jvm");
     reporters = metricManager.getReporters(

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d7bc947/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 27c038b..dfb5a0f 100644
--- a/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java
@@ -19,7 +19,6 @@ 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;
 
@@ -55,6 +54,11 @@ public class SolrMetricsIntegrationTest extends SolrTestCaseJ4 {
 
   private CoreContainer cc;
   private SolrMetricManager metricManager;
+  private String tag;
+
+  private void assertTagged(Map<String, SolrMetricReporter> reporters, String name) {
+    assertTrue("Reporter '" + name + "' missing in " + reporters, reporters.containsKey(name + "@" + tag));
+  }
 
   @Before
   public void beforeTest() throws Exception {
@@ -68,10 +72,13 @@ public class SolrMetricsIntegrationTest extends SolrTestCaseJ4 {
         new TestHarness.TestCoresLocator(DEFAULT_TEST_CORENAME, initCoreDataDir.getAbsolutePath(), "solrconfig.xml", "schema.xml"));
     h.coreName = DEFAULT_TEST_CORENAME;
     metricManager = cc.getMetricManager();
+    tag = h.getCore().getCoreMetricManager().getTag();
     // 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)));
+    for (String r : INITIAL_REPORTERS) {
+      assertTagged(reporters, r);
+    }
     // test rename operation
     cc.rename(DEFAULT_TEST_CORENAME, CORE_NAME);
     h.coreName = CORE_NAME;
@@ -101,7 +108,7 @@ public class SolrMetricsIntegrationTest extends SolrTestCaseJ4 {
     deleteCore();
 
     for (String reporterName : RENAMED_REPORTERS) {
-      SolrMetricReporter reporter = reporters.get(reporterName);
+      SolrMetricReporter reporter = reporters.get(reporterName + "@" + tag);
       MockMetricReporter mockReporter = (MockMetricReporter) reporter;
       assertTrue("Reporter " + reporterName + " was not closed: " + mockReporter, mockReporter.didClose);
     }
@@ -130,7 +137,7 @@ public class SolrMetricsIntegrationTest extends SolrTestCaseJ4 {
     // SPECIFIC and MULTIREGISTRY were skipped because they were
     // specific to collection1
     for (String reporterName : RENAMED_REPORTERS) {
-      SolrMetricReporter reporter = reporters.get(reporterName);
+      SolrMetricReporter reporter = reporters.get(reporterName + "@" + tag);
       assertNotNull("Reporter " + reporterName + " was not found.", reporter);
       assertTrue(reporter instanceof MockMetricReporter);
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d7bc947/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 ea452b2..82b9d58 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
@@ -64,15 +64,17 @@ public class SolrJmxReporterTest extends SolrTestCaseJ4 {
     coreMetricManager = core.getCoreMetricManager();
     metricManager = core.getCoreDescriptor().getCoreContainer().getMetricManager();
     PluginInfo pluginInfo = createReporterPluginInfo();
-    metricManager.loadReporter(coreMetricManager.getRegistryName(), coreMetricManager.getCore().getResourceLoader(), pluginInfo);
+    metricManager.loadReporter(coreMetricManager.getRegistryName(), coreMetricManager.getCore().getResourceLoader(),
+        pluginInfo, coreMetricManager.getTag());
 
     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);
+    String taggedName = reporterName + "@" + coreMetricManager.getTag();
+    assertNotNull("reporter " + taggedName + " not present among " + reporters, reporters.get(taggedName));
+    assertTrue("wrong reporter class: " + reporters.get(taggedName), reporters.get(taggedName) instanceof SolrJmxReporter);
 
-    reporter = (SolrJmxReporter) reporters.get(reporterName);
+    reporter = (SolrJmxReporter) reporters.get(taggedName);
     mBeanServer = reporter.getMBeanServer();
     assertNotNull("MBean server not found.", mBeanServer);
   }
@@ -144,7 +146,8 @@ public class SolrJmxReporterTest extends SolrTestCaseJ4 {
 
     h.getCoreContainer().reload(h.getCore().getName());
     PluginInfo pluginInfo = createReporterPluginInfo();
-    metricManager.loadReporter(coreMetricManager.getRegistryName(), coreMetricManager.getCore().getResourceLoader(), pluginInfo);
+    metricManager.loadReporter(coreMetricManager.getRegistryName(), coreMetricManager.getCore().getResourceLoader(),
+        pluginInfo, String.valueOf(coreMetricManager.getCore().hashCode()));
     coreMetricManager.registerMetricProducer(scope, producer);
 
     objects = mBeanServer.queryMBeans(null, null);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d7bc947/solr/core/src/test/org/apache/solr/metrics/reporters/solr/SolrCloudReportersTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/reporters/solr/SolrCloudReportersTest.java b/solr/core/src/test/org/apache/solr/metrics/reporters/solr/SolrCloudReportersTest.java
new file mode 100644
index 0000000..91952b8
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/metrics/reporters/solr/SolrCloudReportersTest.java
@@ -0,0 +1,163 @@
+/*
+ * 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.solr;
+
+import java.nio.file.Paths;
+import java.util.Map;
+
+import com.codahale.metrics.Metric;
+import org.apache.commons.io.IOUtils;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.metrics.AggregateMetric;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.metrics.SolrMetricReporter;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ *
+ */
+public class SolrCloudReportersTest extends SolrCloudTestCase {
+  int leaderRegistries;
+  int clusterRegistries;
+
+
+  @BeforeClass
+  public static void configureDummyCluster() throws Exception {
+    configureCluster(0).configure();
+  }
+
+  @Before
+  public void closePreviousCluster() throws Exception {
+    shutdownCluster();
+    leaderRegistries = 0;
+    clusterRegistries = 0;
+  }
+
+  @Test
+  public void testExplicitConfiguration() throws Exception {
+    String solrXml = IOUtils.toString(SolrCloudReportersTest.class.getResourceAsStream("/solr/solr-solrreporter.xml"), "UTF-8");
+    configureCluster(2)
+        .withSolrXml(solrXml).configure();
+    cluster.uploadConfigSet(Paths.get(TEST_PATH().toString(), "configsets", "minimal", "conf"), "test");
+    System.out.println("ZK: " + cluster.getZkServer().getZkAddress());
+    CollectionAdminRequest.createCollection("test_collection", "test", 2, 2)
+        .setMaxShardsPerNode(4)
+        .process(cluster.getSolrClient());
+    waitForState("Expected test_collection with 2 shards and 2 replicas", "test_collection", clusterShape(2, 2));
+    Thread.sleep(15000);
+    cluster.getJettySolrRunners().forEach(jetty -> {
+      CoreContainer cc = jetty.getCoreContainer();
+      // verify registry names
+      for (String name : cc.getCoreNames()) {
+        SolrCore core = cc.getCore(name);
+        try {
+          String registryName = core.getCoreMetricManager().getRegistryName();
+          String leaderRegistryName = core.getCoreMetricManager().getLeaderRegistryName();
+          String coreName = core.getName();
+          String collectionName = core.getCoreDescriptor().getCollectionName();
+          String coreNodeName = core.getCoreDescriptor().getCloudDescriptor().getCoreNodeName();
+          String replicaName = coreName.split("_")[3];
+          String shardId = core.getCoreDescriptor().getCloudDescriptor().getShardId();
+
+          assertEquals("solr.core." + collectionName + "." + shardId + "." + replicaName, registryName);
+          assertEquals("solr.collection." + collectionName + "." + shardId + ".leader", leaderRegistryName);
+
+        } finally {
+          if (core != null) {
+            core.close();
+          }
+        }
+      }
+      SolrMetricManager metricManager = cc.getMetricManager();
+      Map<String, SolrMetricReporter> reporters = metricManager.getReporters("solr.cluster");
+      assertEquals(reporters.toString(), 1, reporters.size());
+      SolrMetricReporter reporter = reporters.get("test");
+      assertNotNull(reporter);
+      assertTrue(reporter.toString(), reporter instanceof SolrClusterReporter);
+      SolrClusterReporter sor = (SolrClusterReporter)reporter;
+      assertEquals(5, sor.getPeriod());
+      for (String registryName : metricManager.registryNames(".*\\.shard[0-9]\\.replica.*")) {
+        reporters = metricManager.getReporters(registryName);
+        assertEquals(reporters.toString(), 1, reporters.size());
+        reporter = null;
+        for (String name : reporters.keySet()) {
+          if (name.startsWith("test")) {
+            reporter = reporters.get(name);
+          }
+        }
+        assertNotNull(reporter);
+        assertTrue(reporter.toString(), reporter instanceof SolrShardReporter);
+        SolrShardReporter srr = (SolrShardReporter)reporter;
+        assertEquals(5, srr.getPeriod());
+      }
+      for (String registryName : metricManager.registryNames(".*\\.leader")) {
+        leaderRegistries++;
+        reporters = metricManager.getReporters(registryName);
+        // no reporters registered for leader registry
+        assertEquals(reporters.toString(), 0, reporters.size());
+        // verify specific metrics
+        Map<String, Metric> metrics = metricManager.registry(registryName).getMetrics();
+        String key = "QUERY./select.requests.count";
+        assertTrue(key, metrics.containsKey(key));
+        assertTrue(key, metrics.get(key) instanceof AggregateMetric);
+        key = "UPDATE./update/json.requests.count";
+        assertTrue(key, metrics.containsKey(key));
+        assertTrue(key, metrics.get(key) instanceof AggregateMetric);
+      }
+      if (metricManager.registryNames().contains("solr.cluster")) {
+        clusterRegistries++;
+        Map<String,Metric> metrics = metricManager.registry("solr.cluster").getMetrics();
+        String key = "jvm.memory.heap.init.value";
+        assertTrue(key, metrics.containsKey(key));
+        assertTrue(key, metrics.get(key) instanceof AggregateMetric);
+        key = "leader.test_collection.shard1.UPDATE./update/json.requests.count.max";
+        assertTrue(key, metrics.containsKey(key));
+        assertTrue(key, metrics.get(key) instanceof AggregateMetric);
+      }
+    });
+    assertEquals("leaderRegistries", 2, leaderRegistries);
+    assertEquals("clusterRegistries", 1, clusterRegistries);
+  }
+
+  @Test
+  public void testDefaultPlugins() throws Exception {
+    String solrXml = IOUtils.toString(SolrCloudReportersTest.class.getResourceAsStream("/solr/solr.xml"), "UTF-8");
+    configureCluster(2)
+        .withSolrXml(solrXml).configure();
+    cluster.uploadConfigSet(Paths.get(TEST_PATH().toString(), "configsets", "minimal", "conf"), "test");
+    System.out.println("ZK: " + cluster.getZkServer().getZkAddress());
+    CollectionAdminRequest.createCollection("test_collection", "test", 2, 2)
+        .setMaxShardsPerNode(4)
+        .process(cluster.getSolrClient());
+    waitForState("Expected test_collection with 2 shards and 2 replicas", "test_collection", clusterShape(2, 2));
+    cluster.getJettySolrRunners().forEach(jetty -> {
+      CoreContainer cc = jetty.getCoreContainer();
+      SolrMetricManager metricManager = cc.getMetricManager();
+      Map<String, SolrMetricReporter> reporters = metricManager.getReporters("solr.cluster");
+      assertEquals(reporters.toString(), 0, reporters.size());
+      for (String registryName : metricManager.registryNames(".*\\.shard[0-9]\\.replica.*")) {
+        reporters = metricManager.getReporters(registryName);
+        assertEquals(reporters.toString(), 0, reporters.size());
+      }
+    });
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d7bc947/solr/core/src/test/org/apache/solr/metrics/reporters/solr/SolrShardReporterTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/reporters/solr/SolrShardReporterTest.java b/solr/core/src/test/org/apache/solr/metrics/reporters/solr/SolrShardReporterTest.java
new file mode 100644
index 0000000..9ce3762
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/metrics/reporters/solr/SolrShardReporterTest.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.metrics.reporters.solr;
+
+import java.lang.invoke.MethodHandles;
+import java.util.Map;
+
+import com.codahale.metrics.Metric;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
+import org.apache.solr.cloud.CloudDescriptor;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.CoreDescriptor;
+import org.apache.solr.metrics.AggregateMetric;
+import org.apache.solr.metrics.SolrCoreMetricManager;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ *
+ */
+public class SolrShardReporterTest extends AbstractFullDistribZkTestBase {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public SolrShardReporterTest() {
+    schemaString = "schema15.xml";      // we need a string id
+  }
+
+  @Override
+  public String getSolrXml() {
+    return "solr-solrreporter.xml";
+  }
+
+  @Test
+  public void test() throws Exception {
+    waitForRecoveriesToFinish("control_collection",
+        jettys.get(0).getCoreContainer().getZkController().getZkStateReader(), false);
+    waitForRecoveriesToFinish("collection1",
+        jettys.get(0).getCoreContainer().getZkController().getZkStateReader(), false);
+    printLayout();
+    // wait for at least two reports
+    Thread.sleep(10000);
+    ClusterState state = jettys.get(0).getCoreContainer().getZkController().getClusterState();
+    for (JettySolrRunner jetty : jettys) {
+      CoreContainer cc = jetty.getCoreContainer();
+      SolrMetricManager metricManager = cc.getMetricManager();
+      for (final String coreName : cc.getCoreNames()) {
+        CoreDescriptor cd = cc.getCoreDescriptor(coreName);
+        if (cd.getCloudDescriptor() == null) { // not a cloud collection
+          continue;
+        }
+        CloudDescriptor cloudDesc = cd.getCloudDescriptor();
+        DocCollection docCollection = state.getCollection(cloudDesc.getCollectionName());
+        String replicaName = SolrCoreMetricManager.parseReplicaName(cloudDesc.getCollectionName(), coreName);
+        if (replicaName == null) {
+          replicaName = cloudDesc.getCoreNodeName();
+        }
+        String registryName = SolrCoreMetricManager.createRegistryName(true,
+            cloudDesc.getCollectionName(), cloudDesc.getShardId(), replicaName, null);
+        String leaderRegistryName = SolrCoreMetricManager.createLeaderRegistryName(true,
+            cloudDesc.getCollectionName(), cloudDesc.getShardId());
+        boolean leader = cloudDesc.isLeader();
+        Slice slice = docCollection.getSlice(cloudDesc.getShardId());
+        int numReplicas = slice.getReplicas().size();
+        if (leader) {
+          assertTrue(metricManager.registryNames() + " doesn't contain " + leaderRegistryName,
+              metricManager.registryNames().contains(leaderRegistryName));
+          Map<String, Metric> metrics = metricManager.registry(leaderRegistryName).getMetrics();
+          metrics.forEach((k, v) -> {
+            assertTrue("Unexpected type of " + k + ": " + v.getClass().getName() + ", " + v,
+                v instanceof AggregateMetric);
+            AggregateMetric am = (AggregateMetric)v;
+            if (!k.startsWith("REPLICATION.peerSync")) {
+              assertEquals(coreName + "::" + registryName + "::" + k + ": " + am.toString(), numReplicas, am.size());
+            }
+          });
+        } else {
+          assertFalse(metricManager.registryNames() + " contains " + leaderRegistryName +
+              " but it's not a leader!",
+              metricManager.registryNames().contains(leaderRegistryName));
+          Map<String, Metric> metrics = metricManager.registry(leaderRegistryName).getMetrics();
+          metrics.forEach((k, v) -> {
+            assertTrue("Unexpected type of " + k + ": " + v.getClass().getName() + ", " + v,
+                v instanceof AggregateMetric);
+            AggregateMetric am = (AggregateMetric)v;
+            if (!k.startsWith("REPLICATION.peerSync")) {
+              assertEquals(coreName + "::" + registryName + "::" + k + ": " + am.toString(), 1, am.size());
+            }
+          });
+        }
+        assertTrue(metricManager.registryNames() + " doesn't contain " + registryName,
+            metricManager.registryNames().contains(registryName));
+      }
+    }
+    SolrMetricManager metricManager = controlJetty.getCoreContainer().getMetricManager();
+    assertTrue(metricManager.registryNames().contains("solr.cluster"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d7bc947/solr/core/src/test/org/apache/solr/util/stats/MetricUtilsTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/util/stats/MetricUtilsTest.java b/solr/core/src/test/org/apache/solr/util/stats/MetricUtilsTest.java
index e39ad6e..8717ad6 100644
--- a/solr/core/src/test/org/apache/solr/util/stats/MetricUtilsTest.java
+++ b/solr/core/src/test/org/apache/solr/util/stats/MetricUtilsTest.java
@@ -17,12 +17,20 @@
 
 package org.apache.solr.util.stats;
 
+import java.util.Collections;
+import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.Meter;
+import com.codahale.metrics.MetricFilter;
+import com.codahale.metrics.MetricRegistry;
 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.metrics.AggregateMetric;
 import org.junit.Test;
 
 public class MetricUtilsTest extends SolrTestCaseJ4 {
@@ -36,7 +44,7 @@ public class MetricUtilsTest extends SolrTestCaseJ4 {
       timer.update(Math.abs(random().nextInt()) + 1, TimeUnit.NANOSECONDS);
     }
     // obtain timer metrics
-    NamedList lst = MetricUtils.timerToNamedList(timer);
+    NamedList lst = new NamedList(MetricUtils.timerToMap(timer, false));
     // check that expected metrics were obtained
     assertEquals(14, lst.size());
     final Snapshot snapshot = timer.getSnapshot();
@@ -52,5 +60,49 @@ public class MetricUtilsTest extends SolrTestCaseJ4 {
     assertEquals(MetricUtils.nsToMs(snapshot.get999thPercentile()), lst.get("p999_ms"));
   }
 
+  @Test
+  public void testMetrics() throws Exception {
+    MetricRegistry registry = new MetricRegistry();
+    Counter counter = registry.counter("counter");
+    counter.inc();
+    Timer timer = registry.timer("timer");
+    Timer.Context ctx = timer.time();
+    Thread.sleep(150);
+    ctx.stop();
+    Meter meter = registry.meter("meter");
+    meter.mark();
+    Histogram histogram = registry.histogram("histogram");
+    histogram.update(10);
+    AggregateMetric am = new AggregateMetric();
+    registry.register("aggregate", am);
+    am.set("foo", 10);
+    am.set("bar", 1);
+    am.set("bar", 2);
+    MetricUtils.toNamedMaps(registry, Collections.singletonList(MetricFilter.ALL), MetricFilter.ALL,
+        false, false, (k, v) -> {
+      if (k.startsWith("counter")) {
+        assertEquals(1L, v.get("count"));
+      } else if (k.startsWith("timer")) {
+        assertEquals(1L, v.get("count"));
+        assertTrue(((Number)v.get("min_ms")).intValue() > 100);
+      } else if (k.startsWith("meter")) {
+        assertEquals(1L, v.get("count"));
+      } else if (k.startsWith("histogram")) {
+        assertEquals(1L, v.get("count"));
+      } else if (k.startsWith("aggregate")) {
+        assertEquals(2, v.get("count"));
+        Map<String, Object> values = (Map<String, Object>)v.get("values");
+        assertNotNull(values);
+        assertEquals(2, values.size());
+        Map<String, Object> update = (Map<String, Object>)values.get("foo");
+        assertEquals(10, update.get("value"));
+        assertEquals(1, update.get("updateCount"));
+        update = (Map<String, Object>)values.get("bar");
+        assertEquals(2, update.get("value"));
+        assertEquals(2, update.get("updateCount"));
+      }
+    });
+  }
+
 }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d7bc947/solr/solrj/src/java/org/apache/solr/client/solrj/impl/BinaryRequestWriter.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/BinaryRequestWriter.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/BinaryRequestWriter.java
index 67274c2..310c282 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/BinaryRequestWriter.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/BinaryRequestWriter.java
@@ -112,8 +112,8 @@ public class BinaryRequestWriter extends RequestWriter {
   /*
    * A hack to get access to the protected internal buffer and avoid an additional copy
    */
-  class BAOS extends ByteArrayOutputStream {
-    byte[] getbuf() {
+  public static class BAOS extends ByteArrayOutputStream {
+    public byte[] getbuf() {
       return super.buf;
     }
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d7bc947/solr/solrj/src/java/org/apache/solr/client/solrj/io/SolrClientCache.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/SolrClientCache.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/SolrClientCache.java
index da94162..132a1a8 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/SolrClientCache.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/SolrClientCache.java
@@ -22,6 +22,7 @@ import java.lang.invoke.MethodHandles;
 import java.util.Map;
 import java.util.HashMap;
 
+import org.apache.http.client.HttpClient;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
@@ -38,15 +39,27 @@ public class SolrClientCache implements Serializable {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   private final Map<String, SolrClient> solrClients = new HashMap<>();
+  private final HttpClient httpClient;
+
+  public SolrClientCache() {
+    httpClient = null;
+  }
+
+  public SolrClientCache(HttpClient httpClient) {
+    this.httpClient = httpClient;
+  }
 
   public synchronized CloudSolrClient getCloudSolrClient(String zkHost) {
     CloudSolrClient client;
     if (solrClients.containsKey(zkHost)) {
       client = (CloudSolrClient) solrClients.get(zkHost);
     } else {
-      client = new CloudSolrClient.Builder()
-          .withZkHost(zkHost)
-          .build();
+      CloudSolrClient.Builder builder = new CloudSolrClient.Builder()
+          .withZkHost(zkHost);
+      if (httpClient != null) {
+        builder = builder.withHttpClient(httpClient);
+      }
+      client = builder.build();
       client.connect();
       solrClients.put(zkHost, client);
     }
@@ -59,8 +72,11 @@ public class SolrClientCache implements Serializable {
     if (solrClients.containsKey(host)) {
       client = (HttpSolrClient) solrClients.get(host);
     } else {
-      client = new HttpSolrClient.Builder(host)
-          .build();
+      HttpSolrClient.Builder builder = new HttpSolrClient.Builder(host);
+      if (httpClient != null) {
+        builder = builder.withHttpClient(httpClient);
+      }
+      client = builder.build();
       solrClients.put(host, client);
     }
     return client;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d7bc947/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestCoreAdmin.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestCoreAdmin.java b/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestCoreAdmin.java
index b2174cd..de7c620 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestCoreAdmin.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestCoreAdmin.java
@@ -251,8 +251,8 @@ public class TestCoreAdmin extends AbstractEmbeddedSolrServerTestCase {
 
     // assert initial metrics
     SolrMetricManager metricManager = cores.getMetricManager();
-    String core0RegistryName = SolrCoreMetricManager.createRegistryName(null, "core0");
-    String core1RegistryName = SolrCoreMetricManager.createRegistryName(null, "core1");
+    String core0RegistryName = SolrCoreMetricManager.createRegistryName(false, null, null, null, "core0");
+    String core1RegistryName = SolrCoreMetricManager.createRegistryName(false, null, null,null, "core1");
     MetricRegistry core0Registry = metricManager.registry(core0RegistryName);
     MetricRegistry core1Registry = metricManager.registry(core1RegistryName);
 


[34/43] lucene-solr:feature/autoscaling: LUCENE-7740: Refactor Range Fields to remove Field suffix (e.g., DoubleRange), move InetAddressRange and InetAddressPoint from sandbox to misc module, and refactor all other range fields from sandbox to core.

Posted by sh...@apache.org.
LUCENE-7740: Refactor Range Fields to remove Field suffix (e.g., DoubleRange),
move InetAddressRange and InetAddressPoint from sandbox to misc module, and
refactor all other range fields from sandbox to core.


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

Branch: refs/heads/feature/autoscaling
Commit: d34d81f9af89657fdd4fe0b3174459142955215b
Parents: 182c20c
Author: Nicholas Knize <nk...@gmail.com>
Authored: Mon Mar 13 01:59:04 2017 -0500
Committer: Nicholas Knize <nk...@gmail.com>
Committed: Mon Mar 13 02:22:29 2017 -0500

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   8 +-
 .../org/apache/lucene/document/DoubleRange.java | 271 +++++++++++++++
 .../org/apache/lucene/document/FloatRange.java  | 271 +++++++++++++++
 .../org/apache/lucene/document/IntRange.java    | 276 +++++++++++++++
 .../org/apache/lucene/document/LongRange.java   | 269 ++++++++++++++
 .../apache/lucene/document/RangeFieldQuery.java | 340 ++++++++++++++++++
 .../org/apache/lucene/index/PointValues.java    |   2 +-
 .../search/TestDoubleRangeFieldQueries.java     | 251 ++++++++++++++
 .../search/TestFloatRangeFieldQueries.java      | 251 ++++++++++++++
 .../lucene/search/TestIntRangeFieldQueries.java | 251 ++++++++++++++
 .../search/TestLongRangeFieldQueries.java       | 251 ++++++++++++++
 .../lucene/document/InetAddressPoint.java       | 313 +++++++++++++++++
 .../lucene/document/InetAddressRange.java       | 168 +++++++++
 .../lucene/document/TestInetAddressPoint.java   | 176 ++++++++++
 .../search/TestInetAddressRangeQueries.java     | 215 ++++++++++++
 .../lucene/document/DoubleRangeField.java       | 282 ---------------
 .../apache/lucene/document/FloatRangeField.java | 282 ---------------
 .../lucene/document/InetAddressPoint.java       | 313 -----------------
 .../lucene/document/InetAddressRangeField.java  | 168 ---------
 .../apache/lucene/document/IntRangeField.java   | 282 ---------------
 .../apache/lucene/document/LongRangeField.java  | 280 ---------------
 .../apache/lucene/document/RangeFieldQuery.java | 340 ------------------
 .../org/apache/lucene/document/package.html     |   3 +-
 .../lucene/document/TestDoubleRangeField.java   |  10 +-
 .../lucene/document/TestInetAddressPoint.java   | 176 ----------
 .../search/BaseRangeFieldQueryTestCase.java     | 344 ------------------
 .../search/TestDoubleRangeFieldQueries.java     | 251 --------------
 .../search/TestFloatRangeFieldQueries.java      | 251 --------------
 .../lucene/search/TestIntRangeFieldQueries.java | 251 --------------
 .../lucene/search/TestIpRangeFieldQueries.java  | 220 ------------
 .../search/TestLongRangeFieldQueries.java       | 251 --------------
 .../search/BaseRangeFieldQueryTestCase.java     | 346 +++++++++++++++++++
 32 files changed, 3662 insertions(+), 3701 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index c2fe191..e14ab53 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -84,6 +84,10 @@ Other
 
 API Changes
 
+* LUCENE-7740: Refactor Range Fields to remove Field suffix (e.g., DoubleRange),
+  move InetAddressRange and InetAddressPoint from sandbox to misc module, and
+  refactor all other range fields from sandbox to core. (Nick Knize)
+
 * LUCENE-7624: TermsQuery has been renamed as TermInSetQuery and moved to core.
   (Alan Woodward)
 
@@ -131,8 +135,8 @@ API Changes
 
 New Features
 
-* LUCENE-7738: Add new InetAddressRangeField for indexing and querying
-  InetAddress ranges. (Nick Knize)
+* LUCENE-7738: Add new InetAddressRange for indexing and querying InetAddress
+  ranges. (Nick Knize)
 
 * LUCENE-7449: Add CROSSES relation support to RangeFieldQuery. (Nick Knize)
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/core/src/java/org/apache/lucene/document/DoubleRange.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/DoubleRange.java b/lucene/core/src/java/org/apache/lucene/document/DoubleRange.java
new file mode 100644
index 0000000..90a8eb9
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/document/DoubleRange.java
@@ -0,0 +1,271 @@
+/*
+ * 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.document;
+
+import org.apache.lucene.document.RangeFieldQuery.QueryType;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.NumericUtils;
+
+/**
+ * An indexed Double Range field.
+ * <p>
+ * This field indexes dimensional ranges defined as min/max pairs. It supports
+ * up to a maximum of 4 dimensions (indexed as 8 numeric values). With 1 dimension representing a single double range,
+ * 2 dimensions representing a bounding box, 3 dimensions a bounding cube, and 4 dimensions a tesseract.
+ * <p>
+ * Multiple values for the same field in one document is supported, and open ended ranges can be defined using
+ * {@code Double.NEGATIVE_INFINITY} and {@code Double.POSITIVE_INFINITY}.
+ *
+ * <p>
+ * This field defines the following static factory methods for common search operations over double ranges:
+ * <ul>
+ *   <li>{@link #newIntersectsQuery newIntersectsQuery()} matches ranges that intersect the defined search range.
+ *   <li>{@link #newWithinQuery newWithinQuery()} matches ranges that are within the defined search range.
+ *   <li>{@link #newContainsQuery newContainsQuery()} matches ranges that contain the defined search range.
+ * </ul>
+ */
+public class DoubleRange extends Field {
+  /** stores double values so number of bytes is 8 */
+  public static final int BYTES = Double.BYTES;
+
+  /**
+   * Create a new DoubleRange type, from min/max parallel arrays
+   *
+   * @param name field name. must not be null.
+   * @param min range min values; each entry is the min value for the dimension
+   * @param max range max values; each entry is the max value for the dimension
+   */
+  public DoubleRange(String name, final double[] min, final double[] max) {
+    super(name, getType(min.length));
+    setRangeValues(min, max);
+  }
+
+  /** set the field type */
+  private static FieldType getType(int dimensions) {
+    if (dimensions > 4) {
+      throw new IllegalArgumentException("DoubleRange does not support greater than 4 dimensions");
+    }
+
+    FieldType ft = new FieldType();
+    // dimensions is set as 2*dimension size (min/max per dimension)
+    ft.setDimensions(dimensions*2, BYTES);
+    ft.freeze();
+    return ft;
+  }
+
+  /**
+   * Changes the values of the field.
+   * @param min array of min values. (accepts {@code Double.NEGATIVE_INFINITY})
+   * @param max array of max values. (accepts {@code Double.POSITIVE_INFINITY})
+   * @throws IllegalArgumentException if {@code min} or {@code max} is invalid
+   */
+  public void setRangeValues(double[] min, double[] max) {
+    checkArgs(min, max);
+    if (min.length*2 != type.pointDimensionCount() || max.length*2 != type.pointDimensionCount()) {
+      throw new IllegalArgumentException("field (name=" + name + ") uses " + type.pointDimensionCount()/2
+          + " dimensions; cannot change to (incoming) " + min.length + " dimensions");
+    }
+
+    final byte[] bytes;
+    if (fieldsData == null) {
+      bytes = new byte[BYTES*2*min.length];
+      fieldsData = new BytesRef(bytes);
+    } else {
+      bytes = ((BytesRef)fieldsData).bytes;
+    }
+    verifyAndEncode(min, max, bytes);
+  }
+
+  /** validate the arguments */
+  private static void checkArgs(final double[] min, final double[] max) {
+    if (min == null || max == null || min.length == 0 || max.length == 0) {
+      throw new IllegalArgumentException("min/max range values cannot be null or empty");
+    }
+    if (min.length != max.length) {
+      throw new IllegalArgumentException("min/max ranges must agree");
+    }
+    if (min.length > 4) {
+      throw new IllegalArgumentException("DoubleRange does not support greater than 4 dimensions");
+    }
+  }
+
+  /**
+   * Encodes the min, max ranges into a byte array
+   */
+  private static byte[] encode(double[] min, double[] max) {
+    checkArgs(min, max);
+    byte[] b = new byte[BYTES*2*min.length];
+    verifyAndEncode(min, max, b);
+    return b;
+  }
+
+  /**
+   * encode the ranges into a sortable byte array ({@code Double.NaN} not allowed)
+   * <p>
+   * example for 4 dimensions (8 bytes per dimension value):
+   * minD1 ... minD4 | maxD1 ... maxD4
+   */
+  static void verifyAndEncode(double[] min, double[] max, byte[] bytes) {
+    for (int d=0,i=0,j=min.length*BYTES; d<min.length; ++d, i+=BYTES, j+=BYTES) {
+      if (Double.isNaN(min[d])) {
+        throw new IllegalArgumentException("invalid min value (" + Double.NaN + ")" + " in DoubleRange");
+      }
+      if (Double.isNaN(max[d])) {
+        throw new IllegalArgumentException("invalid max value (" + Double.NaN + ")" + " in DoubleRange");
+      }
+      if (min[d] > max[d]) {
+        throw new IllegalArgumentException("min value (" + min[d] + ") is greater than max value (" + max[d] + ")");
+      }
+      encode(min[d], bytes, i);
+      encode(max[d], bytes, j);
+    }
+  }
+
+  /** encode the given value into the byte array at the defined offset */
+  private static void encode(double val, byte[] bytes, int offset) {
+    NumericUtils.longToSortableBytes(NumericUtils.doubleToSortableLong(val), bytes, offset);
+  }
+
+  /**
+   * Get the min value for the given dimension
+   * @param dimension the dimension, always positive
+   * @return the decoded min value
+   */
+  public double getMin(int dimension) {
+    if (dimension < 0 || dimension >= type.pointDimensionCount()/2) {
+      throw new IllegalArgumentException("dimension request (" + dimension +
+          ") out of bounds for field (name=" + name + " dimensions=" + type.pointDimensionCount()/2 + "). ");
+    }
+    return decodeMin(((BytesRef)fieldsData).bytes, dimension);
+  }
+
+  /**
+   * Get the max value for the given dimension
+   * @param dimension the dimension, always positive
+   * @return the decoded max value
+   */
+  public double getMax(int dimension) {
+    if (dimension < 0 || dimension >= type.pointDimensionCount()/2) {
+      throw new IllegalArgumentException("dimension request (" + dimension +
+          ") out of bounds for field (name=" + name + " dimensions=" + type.pointDimensionCount()/2 + "). ");
+    }
+    return decodeMax(((BytesRef)fieldsData).bytes, dimension);
+  }
+
+  /** decodes the min value (for the defined dimension) from the encoded input byte array */
+  static double decodeMin(byte[] b, int dimension) {
+    int offset = dimension*BYTES;
+    return NumericUtils.sortableLongToDouble(NumericUtils.sortableBytesToLong(b, offset));
+  }
+
+  /** decodes the max value (for the defined dimension) from the encoded input byte array */
+  static double decodeMax(byte[] b, int dimension) {
+    int offset = b.length/2 + dimension*BYTES;
+    return NumericUtils.sortableLongToDouble(NumericUtils.sortableBytesToLong(b, offset));
+  }
+
+  /**
+   * Create a query for matching indexed ranges that intersect the defined range.
+   * @param field field name. must not be null.
+   * @param min array of min values. (accepts {@code Double.NEGATIVE_INFINITY})
+   * @param max array of max values. (accepts {@code Double.POSITIVE_INFINITY})
+   * @return query for matching intersecting ranges (overlap, within, or contains)
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newIntersectsQuery(String field, final double[] min, final double[] max) {
+    return newRelationQuery(field, min, max, QueryType.INTERSECTS);
+  }
+
+  /**
+   * Create a query for matching indexed ranges that contain the defined range.
+   * @param field field name. must not be null.
+   * @param min array of min values. (accepts {@code Double.MIN_VALUE})
+   * @param max array of max values. (accepts {@code Double.MAX_VALUE})
+   * @return query for matching ranges that contain the defined range
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newContainsQuery(String field, final double[] min, final double[] max) {
+    return newRelationQuery(field, min, max, QueryType.CONTAINS);
+  }
+
+  /**
+   * Create a query for matching indexed ranges that are within the defined range.
+   * @param field field name. must not be null.
+   * @param min array of min values. (accepts {@code Double.MIN_VALUE})
+   * @param max array of max values. (accepts {@code Double.MAX_VALUE})
+   * @return query for matching ranges within the defined range
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newWithinQuery(String field, final double[] min, final double[] max) {
+    return newRelationQuery(field, min, max, QueryType.WITHIN);
+  }
+
+  /**
+   * Create a query for matching indexed ranges that cross the defined range.
+   * A CROSSES is defined as any set of ranges that are not disjoint and not wholly contained by
+   * the query. Effectively, its the complement of union(WITHIN, DISJOINT).
+   * @param field field name. must not be null.
+   * @param min array of min values. (accepts {@code Double.MIN_VALUE})
+   * @param max array of max values. (accepts {@code Double.MAX_VALUE})
+   * @return query for matching ranges within the defined range
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newCrossesQuery(String field, final double[] min, final double[] max) {
+    return newRelationQuery(field, min, max, QueryType.CROSSES);
+  }
+
+  /** helper method for creating the desired relational query */
+  private static Query newRelationQuery(String field, final double[] min, final double[] max, QueryType relation) {
+    checkArgs(min, max);
+    return new RangeFieldQuery(field, encode(min, max), min.length, relation) {
+      @Override
+      protected String toString(byte[] ranges, int dimension) {
+        return DoubleRange.toString(ranges, dimension);
+      }
+    };
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append(getClass().getSimpleName());
+    sb.append(" <");
+    sb.append(name);
+    sb.append(':');
+    byte[] b = ((BytesRef)fieldsData).bytes;
+    toString(b, 0);
+    for (int d=1; d<type.pointDimensionCount(); ++d) {
+      sb.append(' ');
+      toString(b, d);
+    }
+    sb.append('>');
+
+    return sb.toString();
+  }
+
+  /**
+   * Returns the String representation for the range at the given dimension
+   * @param ranges the encoded ranges, never null
+   * @param dimension the dimension of interest
+   * @return The string representation for the range at the provided dimension
+   */
+  private static String toString(byte[] ranges, int dimension) {
+    return "[" + Double.toString(decodeMin(ranges, dimension)) + " : "
+        + Double.toString(decodeMax(ranges, dimension)) + "]";
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/core/src/java/org/apache/lucene/document/FloatRange.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/FloatRange.java b/lucene/core/src/java/org/apache/lucene/document/FloatRange.java
new file mode 100644
index 0000000..8b40538
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/document/FloatRange.java
@@ -0,0 +1,271 @@
+/*
+ * 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.document;
+
+import org.apache.lucene.document.RangeFieldQuery.QueryType;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.NumericUtils;
+
+/**
+ * An indexed Float Range field.
+ * <p>
+ * This field indexes dimensional ranges defined as min/max pairs. It supports
+ * up to a maximum of 4 dimensions (indexed as 8 numeric values). With 1 dimension representing a single float range,
+ * 2 dimensions representing a bounding box, 3 dimensions a bounding cube, and 4 dimensions a tesseract.
+ * <p>
+ * Multiple values for the same field in one document is supported, and open ended ranges can be defined using
+ * {@code Float.NEGATIVE_INFINITY} and {@code Float.POSITIVE_INFINITY}.
+ *
+ * <p>
+ * This field defines the following static factory methods for common search operations over float ranges:
+ * <ul>
+ *   <li>{@link #newIntersectsQuery newIntersectsQuery()} matches ranges that intersect the defined search range.
+ *   <li>{@link #newWithinQuery newWithinQuery()} matches ranges that are within the defined search range.
+ *   <li>{@link #newContainsQuery newContainsQuery()} matches ranges that contain the defined search range.
+ * </ul>
+ */
+public class FloatRange extends Field {
+  /** stores float values so number of bytes is 4 */
+  public static final int BYTES = Float.BYTES;
+
+  /**
+   * Create a new FloatRange type, from min/max parallel arrays
+   *
+   * @param name field name. must not be null.
+   * @param min range min values; each entry is the min value for the dimension
+   * @param max range max values; each entry is the max value for the dimension
+   */
+  public FloatRange(String name, final float[] min, final float[] max) {
+    super(name, getType(min.length));
+    setRangeValues(min, max);
+  }
+
+  /** set the field type */
+  private static FieldType getType(int dimensions) {
+    if (dimensions > 4) {
+      throw new IllegalArgumentException("FloatRange does not support greater than 4 dimensions");
+    }
+
+    FieldType ft = new FieldType();
+    // dimensions is set as 2*dimension size (min/max per dimension)
+    ft.setDimensions(dimensions*2, BYTES);
+    ft.freeze();
+    return ft;
+  }
+
+  /**
+   * Changes the values of the field.
+   * @param min array of min values. (accepts {@code Float.NEGATIVE_INFINITY})
+   * @param max array of max values. (accepts {@code Float.POSITIVE_INFINITY})
+   * @throws IllegalArgumentException if {@code min} or {@code max} is invalid
+   */
+  public void setRangeValues(float[] min, float[] max) {
+    checkArgs(min, max);
+    if (min.length*2 != type.pointDimensionCount() || max.length*2 != type.pointDimensionCount()) {
+      throw new IllegalArgumentException("field (name=" + name + ") uses " + type.pointDimensionCount()/2
+          + " dimensions; cannot change to (incoming) " + min.length + " dimensions");
+    }
+
+    final byte[] bytes;
+    if (fieldsData == null) {
+      bytes = new byte[BYTES*2*min.length];
+      fieldsData = new BytesRef(bytes);
+    } else {
+      bytes = ((BytesRef)fieldsData).bytes;
+    }
+    verifyAndEncode(min, max, bytes);
+  }
+
+  /** validate the arguments */
+  private static void checkArgs(final float[] min, final float[] max) {
+    if (min == null || max == null || min.length == 0 || max.length == 0) {
+      throw new IllegalArgumentException("min/max range values cannot be null or empty");
+    }
+    if (min.length != max.length) {
+      throw new IllegalArgumentException("min/max ranges must agree");
+    }
+    if (min.length > 4) {
+      throw new IllegalArgumentException("FloatRange does not support greater than 4 dimensions");
+    }
+  }
+
+  /**
+   * Encodes the min, max ranges into a byte array
+   */
+  private static byte[] encode(float[] min, float[] max) {
+    checkArgs(min, max);
+    byte[] b = new byte[BYTES*2*min.length];
+    verifyAndEncode(min, max, b);
+    return b;
+  }
+
+  /**
+   * encode the ranges into a sortable byte array ({@code Float.NaN} not allowed)
+   * <p>
+   * example for 4 dimensions (8 bytes per dimension value):
+   * minD1 ... minD4 | maxD1 ... maxD4
+   */
+  static void verifyAndEncode(float[] min, float[] max, byte[] bytes) {
+    for (int d=0,i=0,j=min.length*BYTES; d<min.length; ++d, i+=BYTES, j+=BYTES) {
+      if (Double.isNaN(min[d])) {
+        throw new IllegalArgumentException("invalid min value (" + Float.NaN + ")" + " in FloatRange");
+      }
+      if (Double.isNaN(max[d])) {
+        throw new IllegalArgumentException("invalid max value (" + Float.NaN + ")" + " in FloatRange");
+      }
+      if (min[d] > max[d]) {
+        throw new IllegalArgumentException("min value (" + min[d] + ") is greater than max value (" + max[d] + ")");
+      }
+      encode(min[d], bytes, i);
+      encode(max[d], bytes, j);
+    }
+  }
+
+  /** encode the given value into the byte array at the defined offset */
+  private static void encode(float val, byte[] bytes, int offset) {
+    NumericUtils.intToSortableBytes(NumericUtils.floatToSortableInt(val), bytes, offset);
+  }
+
+  /**
+   * Get the min value for the given dimension
+   * @param dimension the dimension, always positive
+   * @return the decoded min value
+   */
+  public float getMin(int dimension) {
+    if (dimension < 0 || dimension >= type.pointDimensionCount()/2) {
+      throw new IllegalArgumentException("dimension request (" + dimension +
+          ") out of bounds for field (name=" + name + " dimensions=" + type.pointDimensionCount()/2 + "). ");
+    }
+    return decodeMin(((BytesRef)fieldsData).bytes, dimension);
+  }
+
+  /**
+   * Get the max value for the given dimension
+   * @param dimension the dimension, always positive
+   * @return the decoded max value
+   */
+  public float getMax(int dimension) {
+    if (dimension < 0 || dimension >= type.pointDimensionCount()/2) {
+      throw new IllegalArgumentException("dimension request (" + dimension +
+          ") out of bounds for field (name=" + name + " dimensions=" + type.pointDimensionCount()/2 + "). ");
+    }
+    return decodeMax(((BytesRef)fieldsData).bytes, dimension);
+  }
+
+  /** decodes the min value (for the defined dimension) from the encoded input byte array */
+  static float decodeMin(byte[] b, int dimension) {
+    int offset = dimension*BYTES;
+    return NumericUtils.sortableIntToFloat(NumericUtils.sortableBytesToInt(b, offset));
+  }
+
+  /** decodes the max value (for the defined dimension) from the encoded input byte array */
+  static float decodeMax(byte[] b, int dimension) {
+    int offset = b.length/2 + dimension*BYTES;
+    return NumericUtils.sortableIntToFloat(NumericUtils.sortableBytesToInt(b, offset));
+  }
+
+  /**
+   * Create a query for matching indexed ranges that intersect the defined range.
+   * @param field field name. must not be null.
+   * @param min array of min values. (accepts {@code Float.NEGATIVE_INFINITY})
+   * @param max array of max values. (accepts {@code Float.MAX_VALUE})
+   * @return query for matching intersecting ranges (overlap, within, or contains)
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newIntersectsQuery(String field, final float[] min, final float[] max) {
+    return newRelationQuery(field, min, max, QueryType.INTERSECTS);
+  }
+
+  /**
+   * Create a query for matching indexed float ranges that contain the defined range.
+   * @param field field name. must not be null.
+   * @param min array of min values. (accepts {@code Float.NEGATIVE_INFINITY})
+   * @param max array of max values. (accepts {@code Float.POSITIVE_INFINITY})
+   * @return query for matching ranges that contain the defined range
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newContainsQuery(String field, final float[] min, final float[] max) {
+    return newRelationQuery(field, min, max, QueryType.CONTAINS);
+  }
+
+  /**
+   * Create a query for matching indexed ranges that are within the defined range.
+   * @param field field name. must not be null.
+   * @param min array of min values. (accepts {@code Float.NEGATIVE_INFINITY})
+   * @param max array of max values. (accepts {@code Float.POSITIVE_INFINITY})
+   * @return query for matching ranges within the defined range
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newWithinQuery(String field, final float[] min, final float[] max) {
+    return newRelationQuery(field, min, max, QueryType.WITHIN);
+  }
+
+  /**
+   * Create a query for matching indexed ranges that cross the defined range.
+   * A CROSSES is defined as any set of ranges that are not disjoint and not wholly contained by
+   * the query. Effectively, its the complement of union(WITHIN, DISJOINT).
+   * @param field field name. must not be null.
+   * @param min array of min values. (accepts {@code Float.NEGATIVE_INFINITY})
+   * @param max array of max values. (accepts {@code Float.POSITIVE_INFINITY})
+   * @return query for matching ranges within the defined range
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newCrossesQuery(String field, final float[] min, final float[] max) {
+    return newRelationQuery(field, min, max, QueryType.CROSSES);
+  }
+
+  /** helper method for creating the desired relational query */
+  private static Query newRelationQuery(String field, final float[] min, final float[] max, QueryType relation) {
+    checkArgs(min, max);
+    return new RangeFieldQuery(field, encode(min, max), min.length, relation) {
+      @Override
+      protected String toString(byte[] ranges, int dimension) {
+        return FloatRange.toString(ranges, dimension);
+      }
+    };
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append(getClass().getSimpleName());
+    sb.append(" <");
+    sb.append(name);
+    sb.append(':');
+    byte[] b = ((BytesRef)fieldsData).bytes;
+    toString(b, 0);
+    for (int d=1; d<type.pointDimensionCount(); ++d) {
+      sb.append(' ');
+      toString(b, d);
+    }
+    sb.append('>');
+
+    return sb.toString();
+  }
+
+  /**
+   * Returns the String representation for the range at the given dimension
+   * @param ranges the encoded ranges, never null
+   * @param dimension the dimension of interest
+   * @return The string representation for the range at the provided dimension
+   */
+  private static String toString(byte[] ranges, int dimension) {
+    return "[" + Float.toString(decodeMin(ranges, dimension)) + " : "
+        + Float.toString(decodeMax(ranges, dimension)) + "]";
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/core/src/java/org/apache/lucene/document/IntRange.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/IntRange.java b/lucene/core/src/java/org/apache/lucene/document/IntRange.java
new file mode 100644
index 0000000..2618f14
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/document/IntRange.java
@@ -0,0 +1,276 @@
+/*
+ * 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.document;
+
+import org.apache.lucene.document.RangeFieldQuery.QueryType;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.NumericUtils;
+
+/**
+ * An indexed Integer Range field.
+ * <p>
+ * This field indexes dimensional ranges defined as min/max pairs. It supports
+ * up to a maximum of 4 dimensions (indexed as 8 numeric values). With 1 dimension representing a single integer range,
+ * 2 dimensions representing a bounding box, 3 dimensions a bounding cube, and 4 dimensions a tesseract.
+ * <p>
+ * Multiple values for the same field in one document is supported, and open ended ranges can be defined using
+ * {@code Integer.MIN_VALUE} and {@code Integer.MAX_VALUE}.
+ *
+ * <p>
+ * This field defines the following static factory methods for common search operations over integer ranges:
+ * <ul>
+ *   <li>{@link #newIntersectsQuery newIntersectsQuery()} matches ranges that intersect the defined search range.
+ *   <li>{@link #newWithinQuery newWithinQuery()} matches ranges that are within the defined search range.
+ *   <li>{@link #newContainsQuery newContainsQuery()} matches ranges that contain the defined search range.
+ * </ul>
+ */
+public class IntRange extends Field {
+  /** stores integer values so number of bytes is 4 */
+  public static final int BYTES = Integer.BYTES;
+
+  /**
+   * Create a new IntRange type, from min/max parallel arrays
+   *
+   * @param name field name. must not be null.
+   * @param min range min values; each entry is the min value for the dimension
+   * @param max range max values; each entry is the max value for the dimension
+   */
+  public IntRange(String name, final int[] min, final int[] max) {
+    super(name, getType(min.length));
+    setRangeValues(min, max);
+  }
+
+  /** set the field type */
+  private static FieldType getType(int dimensions) {
+    if (dimensions > 4) {
+      throw new IllegalArgumentException("IntRange does not support greater than 4 dimensions");
+    }
+
+    FieldType ft = new FieldType();
+    // dimensions is set as 2*dimension size (min/max per dimension)
+    ft.setDimensions(dimensions*2, BYTES);
+    ft.freeze();
+    return ft;
+  }
+
+  /**
+   * Changes the values of the field.
+   * @param min array of min values. (accepts {@code Integer.NEGATIVE_INFINITY})
+   * @param max array of max values. (accepts {@code Integer.POSITIVE_INFINITY})
+   * @throws IllegalArgumentException if {@code min} or {@code max} is invalid
+   */
+  public void setRangeValues(int[] min, int[] max) {
+    checkArgs(min, max);
+    if (min.length*2 != type.pointDimensionCount() || max.length*2 != type.pointDimensionCount()) {
+      throw new IllegalArgumentException("field (name=" + name + ") uses " + type.pointDimensionCount()/2
+          + " dimensions; cannot change to (incoming) " + min.length + " dimensions");
+    }
+
+    final byte[] bytes;
+    if (fieldsData == null) {
+      bytes = new byte[BYTES*2*min.length];
+      fieldsData = new BytesRef(bytes);
+    } else {
+      bytes = ((BytesRef)fieldsData).bytes;
+    }
+    verifyAndEncode(min, max, bytes);
+  }
+
+  /** validate the arguments */
+  private static void checkArgs(final int[] min, final int[] max) {
+    if (min == null || max == null || min.length == 0 || max.length == 0) {
+      throw new IllegalArgumentException("min/max range values cannot be null or empty");
+    }
+    if (min.length != max.length) {
+      throw new IllegalArgumentException("min/max ranges must agree");
+    }
+    if (min.length > 4) {
+      throw new IllegalArgumentException("IntRange does not support greater than 4 dimensions");
+    }
+  }
+
+  /**
+   * Encodes the min, max ranges into a byte array
+   */
+  private static byte[] encode(int[] min, int[] max) {
+    checkArgs(min, max);
+    byte[] b = new byte[BYTES*2*min.length];
+    verifyAndEncode(min, max, b);
+    return b;
+  }
+
+  /**
+   * encode the ranges into a sortable byte array ({@code Double.NaN} not allowed)
+   * <p>
+   * example for 4 dimensions (8 bytes per dimension value):
+   * minD1 ... minD4 | maxD1 ... maxD4
+   */
+  static void verifyAndEncode(int[] min, int[] max, byte[] bytes) {
+    for (int d=0,i=0,j=min.length*BYTES; d<min.length; ++d, i+=BYTES, j+=BYTES) {
+      if (Double.isNaN(min[d])) {
+        throw new IllegalArgumentException("invalid min value (" + Double.NaN + ")" + " in IntRange");
+      }
+      if (Double.isNaN(max[d])) {
+        throw new IllegalArgumentException("invalid max value (" + Double.NaN + ")" + " in IntRange");
+      }
+      if (min[d] > max[d]) {
+        throw new IllegalArgumentException("min value (" + min[d] + ") is greater than max value (" + max[d] + ")");
+      }
+      encode(min[d], bytes, i);
+      encode(max[d], bytes, j);
+    }
+  }
+
+  /** encode the given value into the byte array at the defined offset */
+  private static void encode(int val, byte[] bytes, int offset) {
+    NumericUtils.intToSortableBytes(val, bytes, offset);
+  }
+
+  /**
+   * Get the min value for the given dimension
+   * @param dimension the dimension, always positive
+   * @return the decoded min value
+   */
+  public int getMin(int dimension) {
+    if (dimension < 0 || dimension >= type.pointDimensionCount()/2) {
+      throw new IllegalArgumentException("dimension request (" + dimension +
+          ") out of bounds for field (name=" + name + " dimensions=" + type.pointDimensionCount()/2 + "). ");
+    }
+    return decodeMin(((BytesRef)fieldsData).bytes, dimension);
+  }
+
+  /**
+   * Get the max value for the given dimension
+   * @param dimension the dimension, always positive
+   * @return the decoded max value
+   */
+  public int getMax(int dimension) {
+    if (dimension < 0 || dimension >= type.pointDimensionCount()/2) {
+      throw new IllegalArgumentException("dimension request (" + dimension +
+          ") out of bounds for field (name=" + name + " dimensions=" + type.pointDimensionCount()/2 + "). ");
+    }
+    return decodeMax(((BytesRef)fieldsData).bytes, dimension);
+  }
+
+  /** decodes the min value (for the defined dimension) from the encoded input byte array */
+  static int decodeMin(byte[] b, int dimension) {
+    int offset = dimension*BYTES;
+    return NumericUtils.sortableBytesToInt(b, offset);
+  }
+
+  /** decodes the max value (for the defined dimension) from the encoded input byte array */
+  static int decodeMax(byte[] b, int dimension) {
+    int offset = b.length/2 + dimension*BYTES;
+    return NumericUtils.sortableBytesToInt(b, offset);
+  }
+
+  /**
+   * Create a query for matching indexed ranges that intersect the defined range.
+   * @param field field name. must not be null.
+   * @param min array of min values. (accepts {@code Integer.MIN_VALUE})
+   * @param max array of max values. (accepts {@code Integer.MAX_VALUE})
+   * @return query for matching intersecting ranges (overlap, within, or contains)
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newIntersectsQuery(String field, final int[] min, final int[] max) {
+    return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.INTERSECTS) {
+      @Override
+      protected String toString(byte[] ranges, int dimension) {
+        return IntRange.toString(ranges, dimension);
+      }
+    };
+  }
+
+  /**
+   * Create a query for matching indexed ranges that contain the defined range.
+   * @param field field name. must not be null.
+   * @param min array of min values. (accepts {@code Integer.MIN_VALUE})
+   * @param max array of max values. (accepts {@code Integer.MAX_VALUE})
+   * @return query for matching ranges that contain the defined range
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newContainsQuery(String field, final int[] min, final int[] max) {
+    return newRelationQuery(field, min, max, QueryType.CONTAINS);
+  }
+
+  /**
+   * Create a query for matching indexed ranges that are within the defined range.
+   * @param field field name. must not be null.
+   * @param min array of min values. (accepts {@code Integer.MIN_VALUE})
+   * @param max array of max values. (accepts {@code Integer.MAX_VALUE})
+   * @return query for matching ranges within the defined range
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newWithinQuery(String field, final int[] min, final int[] max) {
+    return newRelationQuery(field, min, max, QueryType.WITHIN);
+  }
+
+  /**
+   * Create a query for matching indexed ranges that cross the defined range.
+   * A CROSSES is defined as any set of ranges that are not disjoint and not wholly contained by
+   * the query. Effectively, its the complement of union(WITHIN, DISJOINT).
+   * @param field field name. must not be null.
+   * @param min array of min values. (accepts {@code Integer.MIN_VALUE})
+   * @param max array of max values. (accepts {@code Integer.MAX_VALUE})
+   * @return query for matching ranges within the defined range
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newCrossesQuery(String field, final int[] min, final int[] max) {
+    return newRelationQuery(field, min, max, QueryType.CROSSES);
+  }
+
+  /** helper method for creating the desired relational query */
+  private static Query newRelationQuery(String field, final int[] min, final int[] max, QueryType relation) {
+    checkArgs(min, max);
+    return new RangeFieldQuery(field, encode(min, max), min.length, relation) {
+      @Override
+      protected String toString(byte[] ranges, int dimension) {
+        return IntRange.toString(ranges, dimension);
+      }
+    };
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append(getClass().getSimpleName());
+    sb.append(" <");
+    sb.append(name);
+    sb.append(':');
+    byte[] b = ((BytesRef)fieldsData).bytes;
+    toString(b, 0);
+    for (int d=1; d<type.pointDimensionCount(); ++d) {
+      sb.append(' ');
+      toString(b, d);
+    }
+    sb.append('>');
+
+    return sb.toString();
+  }
+
+  /**
+   * Returns the String representation for the range at the given dimension
+   * @param ranges the encoded ranges, never null
+   * @param dimension the dimension of interest
+   * @return The string representation for the range at the provided dimension
+   */
+  private static String toString(byte[] ranges, int dimension) {
+    return "[" + Integer.toString(decodeMin(ranges, dimension)) + " : "
+        + Integer.toString(decodeMax(ranges, dimension)) + "]";
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/core/src/java/org/apache/lucene/document/LongRange.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/LongRange.java b/lucene/core/src/java/org/apache/lucene/document/LongRange.java
new file mode 100644
index 0000000..009f4a1
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/document/LongRange.java
@@ -0,0 +1,269 @@
+/*
+ * 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.document;
+
+import org.apache.lucene.document.RangeFieldQuery.QueryType;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.NumericUtils;
+
+/**
+ * An indexed Long Range field.
+ * <p>
+ * This field indexes dimensional ranges defined as min/max pairs. It supports
+ * up to a maximum of 4 dimensions (indexed as 8 numeric values). With 1 dimension representing a single long range,
+ * 2 dimensions representing a bounding box, 3 dimensions a bounding cube, and 4 dimensions a tesseract.
+ * <p>
+ * Multiple values for the same field in one document is supported, and open ended ranges can be defined using
+ * {@code Long.MIN_VALUE} and {@code Long.MAX_VALUE}.
+ *
+ * <p>
+ * This field defines the following static factory methods for common search operations over long ranges:
+ * <ul>
+ *   <li>{@link #newIntersectsQuery newIntersectsQuery()} matches ranges that intersect the defined search range.
+ *   <li>{@link #newWithinQuery newWithinQuery()} matches ranges that are within the defined search range.
+ *   <li>{@link #newContainsQuery newContainsQuery()} matches ranges that contain the defined search range.
+ * </ul>
+ */
+public class LongRange extends Field {
+  /** stores long values so number of bytes is 8 */
+  public static final int BYTES = Long.BYTES;
+
+  /**
+   * Create a new LongRange type, from min/max parallel arrays
+   *
+   * @param name field name. must not be null.
+   * @param min range min values; each entry is the min value for the dimension
+   * @param max range max values; each entry is the max value for the dimension
+   */
+  public LongRange(String name, final long[] min, final long[] max) {
+    super(name, getType(min.length));
+    setRangeValues(min, max);
+  }
+
+  /** set the field type */
+  private static FieldType getType(int dimensions) {
+    if (dimensions > 4) {
+      throw new IllegalArgumentException("LongRange does not support greater than 4 dimensions");
+    }
+
+    FieldType ft = new FieldType();
+    // dimensions is set as 2*dimension size (min/max per dimension)
+    ft.setDimensions(dimensions*2, BYTES);
+    ft.freeze();
+    return ft;
+  }
+
+  /**
+   * Changes the values of the field.
+   * @param min array of min values. (accepts {@code Long.MIN_VALUE})
+   * @param max array of max values. (accepts {@code Long.MAX_VALUE})
+   * @throws IllegalArgumentException if {@code min} or {@code max} is invalid
+   */
+  public void setRangeValues(long[] min, long[] max) {
+    checkArgs(min, max);
+    if (min.length*2 != type.pointDimensionCount() || max.length*2 != type.pointDimensionCount()) {
+      throw new IllegalArgumentException("field (name=" + name + ") uses " + type.pointDimensionCount()/2
+          + " dimensions; cannot change to (incoming) " + min.length + " dimensions");
+    }
+
+    final byte[] bytes;
+    if (fieldsData == null) {
+      bytes = new byte[BYTES*2*min.length];
+      fieldsData = new BytesRef(bytes);
+    } else {
+      bytes = ((BytesRef)fieldsData).bytes;
+    }
+    verifyAndEncode(min, max, bytes);
+  }
+
+  /** validate the arguments */
+  private static void checkArgs(final long[] min, final long[] max) {
+    if (min == null || max == null || min.length == 0 || max.length == 0) {
+      throw new IllegalArgumentException("min/max range values cannot be null or empty");
+    }
+    if (min.length != max.length) {
+      throw new IllegalArgumentException("min/max ranges must agree");
+    }
+    if (min.length > 4) {
+      throw new IllegalArgumentException("LongRange does not support greater than 4 dimensions");
+    }
+  }
+
+  /** Encodes the min, max ranges into a byte array */
+  private static byte[] encode(long[] min, long[] max) {
+    checkArgs(min, max);
+    byte[] b = new byte[BYTES*2*min.length];
+    verifyAndEncode(min, max, b);
+    return b;
+  }
+
+  /**
+   * encode the ranges into a sortable byte array ({@code Double.NaN} not allowed)
+   * <p>
+   * example for 4 dimensions (8 bytes per dimension value):
+   * minD1 ... minD4 | maxD1 ... maxD4
+   */
+  static void verifyAndEncode(long[] min, long[] max, byte[] bytes) {
+    for (int d=0,i=0,j=min.length*BYTES; d<min.length; ++d, i+=BYTES, j+=BYTES) {
+      if (Double.isNaN(min[d])) {
+        throw new IllegalArgumentException("invalid min value (" + Double.NaN + ")" + " in LongRange");
+      }
+      if (Double.isNaN(max[d])) {
+        throw new IllegalArgumentException("invalid max value (" + Double.NaN + ")" + " in LongRange");
+      }
+      if (min[d] > max[d]) {
+        throw new IllegalArgumentException("min value (" + min[d] + ") is greater than max value (" + max[d] + ")");
+      }
+      encode(min[d], bytes, i);
+      encode(max[d], bytes, j);
+    }
+  }
+
+  /** encode the given value into the byte array at the defined offset */
+  private static void encode(long val, byte[] bytes, int offset) {
+    NumericUtils.longToSortableBytes(val, bytes, offset);
+  }
+
+  /**
+   * Get the min value for the given dimension
+   * @param dimension the dimension, always positive
+   * @return the decoded min value
+   */
+  public long getMin(int dimension) {
+    if (dimension < 0 || dimension >= type.pointDimensionCount()/2) {
+      throw new IllegalArgumentException("dimension request (" + dimension +
+          ") out of bounds for field (name=" + name + " dimensions=" + type.pointDimensionCount()/2 + "). ");
+    }
+    return decodeMin(((BytesRef)fieldsData).bytes, dimension);
+  }
+
+  /**
+   * Get the max value for the given dimension
+   * @param dimension the dimension, always positive
+   * @return the decoded max value
+   */
+  public long getMax(int dimension) {
+    if (dimension < 0 || dimension >= type.pointDimensionCount()/2) {
+      throw new IllegalArgumentException("dimension request (" + dimension +
+          ") out of bounds for field (name=" + name + " dimensions=" + type.pointDimensionCount()/2 + "). ");
+    }
+    return decodeMax(((BytesRef)fieldsData).bytes, dimension);
+  }
+
+  /** decodes the min value (for the defined dimension) from the encoded input byte array */
+  static long decodeMin(byte[] b, int dimension) {
+    int offset = dimension*BYTES;
+    return NumericUtils.sortableBytesToLong(b, offset);
+  }
+
+  /** decodes the max value (for the defined dimension) from the encoded input byte array */
+  static long decodeMax(byte[] b, int dimension) {
+    int offset = b.length/2 + dimension*BYTES;
+    return NumericUtils.sortableBytesToLong(b, offset);
+  }
+
+  /**
+   * Create a query for matching indexed ranges that intersect the defined range.
+   * @param field field name. must not be null.
+   * @param min array of min values. (accepts {@code Long.MIN_VALUE})
+   * @param max array of max values. (accepts {@code Long.MAX_VALUE})
+   * @return query for matching intersecting ranges (overlap, within, or contains)
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newIntersectsQuery(String field, final long[] min, final long[] max) {
+    return newRelationQuery(field, min, max, QueryType.INTERSECTS);
+  }
+
+  /**
+   * Create a query for matching indexed ranges that contain the defined range.
+   * @param field field name. must not be null.
+   * @param min array of min values. (accepts {@code Long.MIN_VALUE})
+   * @param max array of max values. (accepts {@code Long.MAX_VALUE})
+   * @return query for matching ranges that contain the defined range
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newContainsQuery(String field, final long[] min, final long[] max) {
+    return newRelationQuery(field, min, max, QueryType.CONTAINS);
+  }
+
+  /**
+   * Create a query for matching indexed ranges that are within the defined range.
+   * @param field field name. must not be null.
+   * @param min array of min values. (accepts {@code Long.MIN_VALUE})
+   * @param max array of max values. (accepts {@code Long.MAX_VALUE})
+   * @return query for matching ranges within the defined range
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newWithinQuery(String field, final long[] min, final long[] max) {
+    return newRelationQuery(field, min, max, QueryType.WITHIN);
+  }
+
+  /**
+   * Create a query for matching indexed ranges that cross the defined range.
+   * A CROSSES is defined as any set of ranges that are not disjoint and not wholly contained by
+   * the query. Effectively, its the complement of union(WITHIN, DISJOINT).
+   * @param field field name. must not be null.
+   * @param min array of min values. (accepts {@code Long.MIN_VALUE})
+   * @param max array of max values. (accepts {@code Long.MAX_VALUE})
+   * @return query for matching ranges within the defined range
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newCrossesQuery(String field, final long[] min, final long[] max) {
+    return newRelationQuery(field, min, max, QueryType.CROSSES);
+  }
+
+  /** helper method for creating the desired relational query */
+  private static Query newRelationQuery(String field, final long[] min, final long[] max, QueryType relation) {
+    checkArgs(min, max);
+    return new RangeFieldQuery(field, encode(min, max), min.length, relation) {
+      @Override
+      protected String toString(byte[] ranges, int dimension) {
+        return LongRange.toString(ranges, dimension);
+      }
+    };
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append(getClass().getSimpleName());
+    sb.append(" <");
+    sb.append(name);
+    sb.append(':');
+    byte[] b = ((BytesRef)fieldsData).bytes;
+    toString(b, 0);
+    for (int d=1; d<type.pointDimensionCount(); ++d) {
+      sb.append(' ');
+      toString(b, d);
+    }
+    sb.append('>');
+
+    return sb.toString();
+  }
+
+  /**
+   * Returns the String representation for the range at the given dimension
+   * @param ranges the encoded ranges, never null
+   * @param dimension the dimension of interest
+   * @return The string representation for the range at the provided dimension
+   */
+  private static String toString(byte[] ranges, int dimension) {
+    return "[" + Long.toString(decodeMin(ranges, dimension)) + " : "
+        + Long.toString(decodeMax(ranges, dimension)) + "]";
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/core/src/java/org/apache/lucene/document/RangeFieldQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/RangeFieldQuery.java b/lucene/core/src/java/org/apache/lucene/document/RangeFieldQuery.java
new file mode 100644
index 0000000..10f10fa
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/document/RangeFieldQuery.java
@@ -0,0 +1,340 @@
+/*
+ * 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.document;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Objects;
+import java.util.function.IntPredicate;
+import java.util.function.Predicate;
+
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.index.PointValues.Relation;
+import org.apache.lucene.index.PointValues.IntersectVisitor;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSet;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.DocIdSetBuilder;
+import org.apache.lucene.util.StringHelper;
+
+/**
+ * Query class for searching {@code RangeField} types by a defined {@link Relation}.
+ */
+abstract class RangeFieldQuery extends Query {
+  /** field name */
+  final String field;
+  /** query relation
+   * intersects: {@code CELL_CROSSES_QUERY},
+   * contains: {@code CELL_CONTAINS_QUERY},
+   * within: {@code CELL_WITHIN_QUERY} */
+  final QueryType queryType;
+  /** number of dimensions - max 4 */
+  final int numDims;
+  /** ranges encoded as a sortable byte array */
+  final byte[] ranges;
+  /** number of bytes per dimension */
+  final int bytesPerDim;
+
+  /** Used by {@code RangeFieldQuery} to check how each internal or leaf node relates to the query. */
+  enum QueryType {
+    /** Use this for intersects queries. */
+    INTERSECTS,
+    /** Use this for within queries. */
+    WITHIN,
+    /** Use this for contains */
+    CONTAINS,
+    /** Use this for crosses queries */
+    CROSSES
+  }
+
+  /**
+   * Create a query for searching indexed ranges that match the provided relation.
+   * @param field field name. must not be null.
+   * @param ranges encoded range values; this is done by the {@code RangeField} implementation
+   * @param queryType the query relation
+   */
+  RangeFieldQuery(String field, final byte[] ranges, final int numDims, final QueryType queryType) {
+    checkArgs(field, ranges, numDims);
+    if (queryType == null) {
+      throw new IllegalArgumentException("Query type cannot be null");
+    }
+    this.field = field;
+    this.queryType = queryType;
+    this.numDims = numDims;
+    this.ranges = ranges;
+    this.bytesPerDim = ranges.length / (2*numDims);
+  }
+
+  /** check input arguments */
+  private static void checkArgs(String field, final byte[] ranges, final int numDims) {
+    if (field == null) {
+      throw new IllegalArgumentException("field must not be null");
+    }
+    if (numDims > 4) {
+      throw new IllegalArgumentException("dimension size cannot be greater than 4");
+    }
+    if (ranges == null || ranges.length == 0) {
+      throw new IllegalArgumentException("encoded ranges cannot be null or empty");
+    }
+  }
+
+  /** Check indexed field info against the provided query data. */
+  private void checkFieldInfo(FieldInfo fieldInfo) {
+    if (fieldInfo.getPointDimensionCount()/2 != numDims) {
+      throw new IllegalArgumentException("field=\"" + field + "\" was indexed with numDims="
+          + fieldInfo.getPointDimensionCount()/2 + " but this query has numDims=" + numDims);
+    }
+  }
+
+  @Override
+  public final Weight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException {
+    return new ConstantScoreWeight(this, boost) {
+      final RangeFieldComparator target = new RangeFieldComparator();
+      private DocIdSet buildMatchingDocIdSet(LeafReader reader, PointValues values) throws IOException {
+        DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc(), values, field);
+        values.intersect(
+            new IntersectVisitor() {
+              DocIdSetBuilder.BulkAdder adder;
+              @Override
+              public void grow(int count) {
+                adder = result.grow(count);
+              }
+              @Override
+              public void visit(int docID) throws IOException {
+                adder.add(docID);
+              }
+              @Override
+              public void visit(int docID, byte[] leaf) throws IOException {
+                if (target.matches(leaf)) {
+                  adder.add(docID);
+                }
+              }
+              @Override
+              public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+                byte[] node = getInternalRange(minPackedValue, maxPackedValue);
+                // compute range relation for BKD traversal
+                if (target.intersects(node) == false) {
+                  return Relation.CELL_OUTSIDE_QUERY;
+                } else if (target.within(node)) {
+                  // target within cell; continue traversing:
+                  return Relation.CELL_CROSSES_QUERY;
+                } else if (target.contains(node)) {
+                  // target contains cell; add iff queryType is not a CONTAINS or CROSSES query:
+                  return (queryType == QueryType.CONTAINS || queryType == QueryType.CROSSES) ?
+                      Relation.CELL_OUTSIDE_QUERY : Relation.CELL_INSIDE_QUERY;
+                }
+                // target intersects cell; continue traversing:
+                return Relation.CELL_CROSSES_QUERY;
+              }
+            });
+        return result.build();
+      }
+
+      @Override
+      public Scorer scorer(LeafReaderContext context) throws IOException {
+        LeafReader reader = context.reader();
+        PointValues values = reader.getPointValues(field);
+        if (values == null) {
+          // no docs in this segment indexed any ranges
+          return null;
+        }
+        FieldInfo fieldInfo = reader.getFieldInfos().fieldInfo(field);
+        if (fieldInfo == null) {
+          // no docs in this segment indexed this field
+          return null;
+        }
+        checkFieldInfo(fieldInfo);
+        boolean allDocsMatch = true;
+        if (values.getDocCount() == reader.maxDoc()) {
+          // if query crosses, docs need to be further scrutinized
+          byte[] range = getInternalRange(values.getMinPackedValue(), values.getMaxPackedValue());
+          // if the internal node is not equal and not contained by the query, all docs do not match
+          if (queryType == QueryType.CROSSES || (!Arrays.equals(ranges, range)
+              && (target.contains(range) == false || queryType != QueryType.WITHIN))) {
+            allDocsMatch = false;
+          }
+        } else {
+          allDocsMatch = false;
+        }
+
+        DocIdSetIterator iterator = allDocsMatch == true ?
+            DocIdSetIterator.all(reader.maxDoc()) : buildMatchingDocIdSet(reader, values).iterator();
+        return new ConstantScoreScorer(this, score(), iterator);
+      }
+
+      /** get an encoded byte representation of the internal node; this is
+       *  the lower half of the min array and the upper half of the max array */
+      private byte[] getInternalRange(byte[] min, byte[] max) {
+        byte[] range = new byte[min.length];
+        final int dimSize = numDims * bytesPerDim;
+        System.arraycopy(min, 0, range, 0, dimSize);
+        System.arraycopy(max, dimSize, range, dimSize, dimSize);
+        return range;
+      }
+    };
+  }
+
+  /**
+   * RangeFieldComparator class provides the core comparison logic for accepting or rejecting indexed
+   * {@code RangeField} types based on the defined query range and relation.
+   */
+  class RangeFieldComparator {
+    final Predicate<byte[]> predicate;
+
+    /** constructs the comparator based on the query type */
+    RangeFieldComparator() {
+      switch (queryType) {
+        case INTERSECTS:
+          predicate = this::intersects;
+          break;
+        case WITHIN:
+          predicate = this::contains;
+          break;
+        case CONTAINS:
+          predicate = this::within;
+          break;
+        case CROSSES:
+          // crosses first checks intersection (disjoint automatic fails),
+          // then ensures the query doesn't wholly contain the leaf:
+          predicate = (byte[] leaf) -> this.intersects(leaf)
+              && this.contains(leaf) == false;
+          break;
+        default:
+          throw new IllegalArgumentException("invalid queryType [" + queryType + "] found.");
+      }
+    }
+
+    /** determines if the candidate range matches the query request */
+    private boolean matches(final byte[] candidate) {
+      return (Arrays.equals(ranges, candidate) && queryType != QueryType.CROSSES)
+          || predicate.test(candidate);
+    }
+
+    /** check if query intersects candidate range */
+    private boolean intersects(final byte[] candidate) {
+      return relate((int d) -> compareMinMax(candidate, d) > 0 || compareMaxMin(candidate, d) < 0);
+    }
+
+    /** check if query is within candidate range */
+    private boolean within(final byte[] candidate) {
+      return relate((int d) -> compareMinMin(candidate, d) < 0 || compareMaxMax(candidate, d) > 0);
+    }
+
+    /** check if query contains candidate range */
+    private boolean contains(final byte[] candidate) {
+      return relate((int d) -> compareMinMin(candidate, d) > 0 || compareMaxMax(candidate, d) < 0);
+    }
+
+    /** internal method used by each relation method to test range relation logic */
+    private boolean relate(IntPredicate predicate) {
+      for (int d=0; d<numDims; ++d) {
+        if (predicate.test(d)) {
+          return false;
+        }
+      }
+      return true;
+    }
+
+    /** compare the encoded min value (for the defined query dimension) with the encoded min value in the byte array */
+    private int compareMinMin(byte[] b, int dimension) {
+      // convert dimension to offset:
+      dimension *= bytesPerDim;
+      return StringHelper.compare(bytesPerDim, ranges, dimension, b, dimension);
+    }
+
+    /** compare the encoded min value (for the defined query dimension) with the encoded max value in the byte array */
+    private int compareMinMax(byte[] b, int dimension) {
+      // convert dimension to offset:
+      dimension *= bytesPerDim;
+      return StringHelper.compare(bytesPerDim, ranges, dimension, b, numDims * bytesPerDim + dimension);
+    }
+
+    /** compare the encoded max value (for the defined query dimension) with the encoded min value in the byte array */
+    private int compareMaxMin(byte[] b, int dimension) {
+      // convert dimension to offset:
+      dimension *= bytesPerDim;
+      return StringHelper.compare(bytesPerDim, ranges, numDims * bytesPerDim + dimension, b, dimension);
+    }
+
+    /** compare the encoded max value (for the defined query dimension) with the encoded max value in the byte array */
+    private int compareMaxMax(byte[] b, int dimension) {
+      // convert dimension to max offset:
+      dimension = numDims * bytesPerDim + dimension * bytesPerDim;
+      return StringHelper.compare(bytesPerDim, ranges, dimension, b, dimension);
+    }
+  }
+
+  @Override
+  public int hashCode() {
+    int hash = classHash();
+    hash = 31 * hash + field.hashCode();
+    hash = 31 * hash + numDims;
+    hash = 31 * hash + queryType.hashCode();
+    hash = 31 * hash + Arrays.hashCode(ranges);
+
+    return hash;
+  }
+
+  @Override
+  public final boolean equals(Object o) {
+    return sameClassAs(o) &&
+        equalsTo(getClass().cast(o));
+  }
+
+  protected boolean equalsTo(RangeFieldQuery other) {
+    return Objects.equals(field, other.field) &&
+        numDims == other.numDims &&
+        Arrays.equals(ranges, other.ranges) &&
+        other.queryType == queryType;
+  }
+
+  @Override
+  public String toString(String field) {
+    StringBuilder sb = new StringBuilder();
+    if (this.field.equals(field) == false) {
+      sb.append(this.field);
+      sb.append(':');
+    }
+    sb.append("<ranges:");
+    sb.append(toString(ranges, 0));
+    for (int d=1; d<numDims; ++d) {
+      sb.append(' ');
+      sb.append(toString(ranges, d));
+    }
+    sb.append('>');
+
+    return sb.toString();
+  }
+
+  /**
+   * Returns a string of a single value in a human-readable format for debugging.
+   * This is used by {@link #toString()}.
+   *
+   * @param dimension dimension of the particular value
+   * @param ranges encoded ranges, never null
+   * @return human readable value for debugging
+   */
+  protected abstract String toString(byte[] ranges, int dimension);
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/core/src/java/org/apache/lucene/index/PointValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/PointValues.java b/lucene/core/src/java/org/apache/lucene/index/PointValues.java
index dab9140..186dbd5 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PointValues.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PointValues.java
@@ -46,7 +46,7 @@ import org.apache.lucene.util.bkd.BKDWriter;
  *   <tr><td>{@code double}</td><td>{@link DoublePoint}</td></tr>
  *   <tr><td>{@code byte[]}</td><td>{@link BinaryPoint}</td></tr>
  *   <tr><td>{@link BigInteger}</td><td><a href="{@docRoot}/../sandbox/org/apache/lucene/document/BigIntegerPoint.html">BigIntegerPoint</a>*</td></tr>
- *   <tr><td>{@link InetAddress}</td><td><a href="{@docRoot}/../sandbox/org/apache/lucene/document/InetAddressPoint.html">InetAddressPoint</a>*</td></tr>
+ *   <tr><td>{@link InetAddress}</td><td><a href="{@docRoot}/../misc/org/apache/lucene/document/InetAddressPoint.html">InetAddressPoint</a>*</td></tr>
  * </table>
  * * in the <i>lucene-sandbox</i> jar<br>
  * <p>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/core/src/test/org/apache/lucene/search/TestDoubleRangeFieldQueries.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestDoubleRangeFieldQueries.java b/lucene/core/src/test/org/apache/lucene/search/TestDoubleRangeFieldQueries.java
new file mode 100644
index 0000000..49ca710
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/TestDoubleRangeFieldQueries.java
@@ -0,0 +1,251 @@
+/*
+ * 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 org.apache.lucene.document.Document;
+import org.apache.lucene.document.DoubleRange;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.store.Directory;
+
+/**
+ * Random testing for RangeFieldQueries.
+ */
+public class TestDoubleRangeFieldQueries extends BaseRangeFieldQueryTestCase {
+  private static final String FIELD_NAME = "doubleRangeField";
+
+  private double nextDoubleInternal() {
+    if (rarely()) {
+      return random().nextBoolean() ? Double.POSITIVE_INFINITY : Double.NEGATIVE_INFINITY;
+    }
+    double max = Double.MAX_VALUE / 2;
+    return (max + max) * random().nextDouble() - max;
+  }
+
+  @Override
+  protected Range nextRange(int dimensions) throws Exception {
+    double[] min = new double[dimensions];
+    double[] max = new double[dimensions];
+
+    double minV, maxV;
+    for (int d=0; d<dimensions; ++d) {
+      minV = nextDoubleInternal();
+      maxV = nextDoubleInternal();
+      min[d] = Math.min(minV, maxV);
+      max[d] = Math.max(minV, maxV);
+    }
+    return new DoubleTestRange(min, max);
+  }
+
+  @Override
+  protected DoubleRange newRangeField(Range r) {
+    return new DoubleRange(FIELD_NAME, ((DoubleTestRange)r).min, ((DoubleTestRange)r).max);
+  }
+
+  @Override
+  protected Query newIntersectsQuery(Range r) {
+    return DoubleRange.newIntersectsQuery(FIELD_NAME, ((DoubleTestRange)r).min, ((DoubleTestRange)r).max);
+  }
+
+  @Override
+  protected Query newContainsQuery(Range r) {
+    return DoubleRange.newContainsQuery(FIELD_NAME, ((DoubleTestRange)r).min, ((DoubleTestRange)r).max);
+  }
+
+  @Override
+  protected Query newWithinQuery(Range r) {
+    return DoubleRange.newWithinQuery(FIELD_NAME, ((DoubleTestRange)r).min, ((DoubleTestRange)r).max);
+  }
+
+  @Override
+  protected Query newCrossesQuery(Range r) {
+    return DoubleRange.newCrossesQuery(FIELD_NAME, ((DoubleTestRange)r).min, ((DoubleTestRange)r).max);
+  }
+
+  /** Basic test */
+  public void testBasics() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+
+    // intersects (within)
+    Document document = new Document();
+    document.add(new DoubleRange(FIELD_NAME, new double[] {-10.0, -10.0}, new double[] {9.1, 10.1}));
+    writer.addDocument(document);
+
+    // intersects (crosses)
+    document = new Document();
+    document.add(new DoubleRange(FIELD_NAME, new double[] {10.0, -10.0}, new double[] {20.0, 10.0}));
+    writer.addDocument(document);
+
+    // intersects (contains, crosses)
+    document = new Document();
+    document.add(new DoubleRange(FIELD_NAME, new double[] {-20.0, -20.0}, new double[] {30.0, 30.1}));
+    writer.addDocument(document);
+
+    // intersects (crosses)
+    document = new Document();
+    document.add(new DoubleRange(FIELD_NAME, new double[] {-11.1, -11.2}, new double[] {1.23, 11.5}));
+    writer.addDocument(document);
+
+    // intersects (crosses)
+    document = new Document();
+    document.add(new DoubleRange(FIELD_NAME, new double[] {12.33, 1.2}, new double[] {15.1, 29.9}));
+    writer.addDocument(document);
+
+    // disjoint
+    document = new Document();
+    document.add(new DoubleRange(FIELD_NAME, new double[] {-122.33, 1.2}, new double[] {-115.1, 29.9}));
+    writer.addDocument(document);
+
+    // intersects (crosses)
+    document = new Document();
+    document.add(new DoubleRange(FIELD_NAME, new double[] {Double.NEGATIVE_INFINITY, 1.2}, new double[] {-11.0, 29.9}));
+    writer.addDocument(document);
+
+    // equal (within, contains, intersects)
+    document = new Document();
+    document.add(new DoubleRange(FIELD_NAME, new double[] {-11, -15}, new double[] {15, 20}));
+    writer.addDocument(document);
+
+    // search
+    IndexReader reader = writer.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+    assertEquals(7, searcher.count(DoubleRange.newIntersectsQuery(FIELD_NAME,
+        new double[] {-11.0, -15.0}, new double[] {15.0, 20.0})));
+    assertEquals(2, searcher.count(DoubleRange.newWithinQuery(FIELD_NAME,
+        new double[] {-11.0, -15.0}, new double[] {15.0, 20.0})));
+    assertEquals(2, searcher.count(DoubleRange.newContainsQuery(FIELD_NAME,
+        new double[] {-11.0, -15.0}, new double[] {15.0, 20.0})));
+    assertEquals(5, searcher.count(DoubleRange.newCrossesQuery(FIELD_NAME,
+        new double[] {-11.0, -15.0}, new double[] {15.0, 20.0})));
+
+    reader.close();
+    writer.close();
+    dir.close();
+  }
+
+  /** DoubleRange test class implementation - use to validate DoubleRange */
+  private class DoubleTestRange extends Range {
+    double[] min;
+    double[] max;
+
+    DoubleTestRange(double[] min, double[] max) {
+      assert min != null && max != null && min.length > 0 && max.length > 0
+          : "test box: min/max cannot be null or empty";
+      assert min.length == max.length : "test box: min/max length do not agree";
+      this.min = min;
+      this.max = max;
+    }
+
+    @Override
+    protected int numDimensions() {
+      return min.length;
+    }
+
+    @Override
+    protected Double getMin(int dim) {
+      return min[dim];
+    }
+
+    @Override
+    protected void setMin(int dim, Object val) {
+      double v = (Double)val;
+      if (min[dim] < v) {
+        max[dim] = v;
+      } else {
+        min[dim] = v;
+      }
+    }
+
+    @Override
+    protected Double getMax(int dim) {
+      return max[dim];
+    }
+
+    @Override
+    protected void setMax(int dim, Object val) {
+      double v = (Double)val;
+      if (max[dim] > v) {
+        min[dim] = v;
+      } else {
+        max[dim] = v;
+      }
+    }
+
+    @Override
+    protected boolean isEqual(Range other) {
+      DoubleTestRange o = (DoubleTestRange)other;
+      return Arrays.equals(min, o.min) && Arrays.equals(max, o.max);
+    }
+
+    @Override
+    protected boolean isDisjoint(Range o) {
+      DoubleTestRange other = (DoubleTestRange)o;
+      for (int d=0; d<this.min.length; ++d) {
+        if (this.min[d] > other.max[d] || this.max[d] < other.min[d]) {
+          // disjoint:
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    protected boolean isWithin(Range o) {
+      DoubleTestRange other = (DoubleTestRange)o;
+      for (int d=0; d<this.min.length; ++d) {
+        if ((this.min[d] >= other.min[d] && this.max[d] <= other.max[d]) == false) {
+          // not within:
+          return false;
+        }
+      }
+      return true;
+    }
+
+    @Override
+    protected boolean contains(Range o) {
+      DoubleTestRange other = (DoubleTestRange) o;
+      for (int d=0; d<this.min.length; ++d) {
+        if ((this.min[d] <= other.min[d] && this.max[d] >= other.max[d]) == false) {
+          // not contains:
+          return false;
+        }
+      }
+      return true;
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder b = new StringBuilder();
+      b.append("Box(");
+      b.append(min[0]);
+      b.append(" TO ");
+      b.append(max[0]);
+      for (int d=1; d<min.length; ++d) {
+        b.append(", ");
+        b.append(min[d]);
+        b.append(" TO ");
+        b.append(max[d]);
+      }
+      b.append(")");
+
+      return b.toString();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/core/src/test/org/apache/lucene/search/TestFloatRangeFieldQueries.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestFloatRangeFieldQueries.java b/lucene/core/src/test/org/apache/lucene/search/TestFloatRangeFieldQueries.java
new file mode 100644
index 0000000..6dc5907
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/TestFloatRangeFieldQueries.java
@@ -0,0 +1,251 @@
+/*
+ * 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 org.apache.lucene.document.Document;
+import org.apache.lucene.document.FloatRange;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.store.Directory;
+
+/**
+ * Random testing for FloatRange Queries.
+ */
+public class TestFloatRangeFieldQueries extends BaseRangeFieldQueryTestCase {
+  private static final String FIELD_NAME = "floatRangeField";
+
+  private float nextFloatInternal() {
+    if (rarely()) {
+      return random().nextBoolean() ? Float.NEGATIVE_INFINITY : Float.POSITIVE_INFINITY;
+    }
+    float max = Float.MAX_VALUE / 2;
+    return (max + max) * random().nextFloat() - max;
+  }
+
+  @Override
+  protected Range nextRange(int dimensions) throws Exception {
+    float[] min = new float[dimensions];
+    float[] max = new float[dimensions];
+
+    float minV, maxV;
+    for (int d=0; d<dimensions; ++d) {
+      minV = nextFloatInternal();
+      maxV = nextFloatInternal();
+      min[d] = Math.min(minV, maxV);
+      max[d] = Math.max(minV, maxV);
+    }
+    return new FloatTestRange(min, max);
+  }
+
+  @Override
+  protected FloatRange newRangeField(Range r) {
+    return new FloatRange(FIELD_NAME, ((FloatTestRange)r).min, ((FloatTestRange)r).max);
+  }
+
+  @Override
+  protected Query newIntersectsQuery(Range r) {
+    return FloatRange.newIntersectsQuery(FIELD_NAME, ((FloatTestRange)r).min, ((FloatTestRange)r).max);
+  }
+
+  @Override
+  protected Query newContainsQuery(Range r) {
+    return FloatRange.newContainsQuery(FIELD_NAME, ((FloatTestRange)r).min, ((FloatTestRange)r).max);
+  }
+
+  @Override
+  protected Query newWithinQuery(Range r) {
+    return FloatRange.newWithinQuery(FIELD_NAME, ((FloatTestRange)r).min, ((FloatTestRange)r).max);
+  }
+
+  @Override
+  protected Query newCrossesQuery(Range r) {
+    return FloatRange.newCrossesQuery(FIELD_NAME, ((FloatTestRange)r).min, ((FloatTestRange)r).max);
+  }
+
+  /** Basic test */
+  public void testBasics() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+
+    // intersects (within)
+    Document document = new Document();
+    document.add(new FloatRange(FIELD_NAME, new float[] {-10.0f, -10.0f}, new float[] {9.1f, 10.1f}));
+    writer.addDocument(document);
+
+    // intersects (crosses)
+    document = new Document();
+    document.add(new FloatRange(FIELD_NAME, new float[] {10.0f, -10.0f}, new float[] {20.0f, 10.0f}));
+    writer.addDocument(document);
+
+    // intersects (contains, crosses)
+    document = new Document();
+    document.add(new FloatRange(FIELD_NAME, new float[] {-20.0f, -20.0f}, new float[] {30.0f, 30.1f}));
+    writer.addDocument(document);
+
+    // intersects (crosses)
+    document = new Document();
+    document.add(new FloatRange(FIELD_NAME, new float[] {-11.1f, -11.2f}, new float[] {1.23f, 11.5f}));
+    writer.addDocument(document);
+
+    // intersects (crosses)
+    document = new Document();
+    document.add(new FloatRange(FIELD_NAME, new float[] {12.33f, 1.2f}, new float[] {15.1f, 29.9f}));
+    writer.addDocument(document);
+
+    // disjoint
+    document = new Document();
+    document.add(new FloatRange(FIELD_NAME, new float[] {-122.33f, 1.2f}, new float[] {-115.1f, 29.9f}));
+    writer.addDocument(document);
+
+    // intersects (crosses)
+    document = new Document();
+    document.add(new FloatRange(FIELD_NAME, new float[] {Float.NEGATIVE_INFINITY, 1.2f}, new float[] {-11.0f, 29.9f}));
+    writer.addDocument(document);
+
+    // equal (within, contains, intersects)
+    document = new Document();
+    document.add(new FloatRange(FIELD_NAME, new float[] {-11f, -15f}, new float[] {15f, 20f}));
+    writer.addDocument(document);
+
+    // search
+    IndexReader reader = writer.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+    assertEquals(7, searcher.count(FloatRange.newIntersectsQuery(FIELD_NAME,
+        new float[] {-11.0f, -15.0f}, new float[] {15.0f, 20.0f})));
+    assertEquals(2, searcher.count(FloatRange.newWithinQuery(FIELD_NAME,
+        new float[] {-11.0f, -15.0f}, new float[] {15.0f, 20.0f})));
+    assertEquals(2, searcher.count(FloatRange.newContainsQuery(FIELD_NAME,
+        new float[] {-11.0f, -15.0f}, new float[] {15.0f, 20.0f})));
+    assertEquals(5, searcher.count(FloatRange.newCrossesQuery(FIELD_NAME,
+        new float[] {-11.0f, -15.0f}, new float[] {15.0f, 20.0f})));
+
+    reader.close();
+    writer.close();
+    dir.close();
+  }
+
+  /** FloatRange test class implementation - use to validate FloatRange */
+  private class FloatTestRange extends Range {
+    float[] min;
+    float[] max;
+
+    FloatTestRange(float[] min, float[] max) {
+      assert min != null && max != null && min.length > 0 && max.length > 0
+          : "test box: min/max cannot be null or empty";
+      assert min.length == max.length : "test box: min/max length do not agree";
+      this.min = min;
+      this.max = max;
+    }
+
+    @Override
+    protected int numDimensions() {
+      return min.length;
+    }
+
+    @Override
+    protected Float getMin(int dim) {
+      return min[dim];
+    }
+
+    @Override
+    protected void setMin(int dim, Object val) {
+      float v = (Float)val;
+      if (min[dim] < v) {
+        max[dim] = v;
+      } else {
+        min[dim] = v;
+      }
+    }
+
+    @Override
+    protected Float getMax(int dim) {
+      return max[dim];
+    }
+
+    @Override
+    protected void setMax(int dim, Object val) {
+      float v = (Float)val;
+      if (max[dim] > v) {
+        min[dim] = v;
+      } else {
+        max[dim] = v;
+      }
+    }
+
+    @Override
+    protected boolean isEqual(Range other) {
+      FloatTestRange o = (FloatTestRange)other;
+      return Arrays.equals(min, o.min) && Arrays.equals(max, o.max);
+    }
+
+    @Override
+    protected boolean isDisjoint(Range o) {
+      FloatTestRange other = (FloatTestRange)o;
+      for (int d=0; d<this.min.length; ++d) {
+        if (this.min[d] > other.max[d] || this.max[d] < other.min[d]) {
+          // disjoint:
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    protected boolean isWithin(Range o) {
+      FloatTestRange other = (FloatTestRange)o;
+      for (int d=0; d<this.min.length; ++d) {
+        if ((this.min[d] >= other.min[d] && this.max[d] <= other.max[d]) == false) {
+          // not within:
+          return false;
+        }
+      }
+      return true;
+    }
+
+    @Override
+    protected boolean contains(Range o) {
+      FloatTestRange other = (FloatTestRange) o;
+      for (int d=0; d<this.min.length; ++d) {
+        if ((this.min[d] <= other.min[d] && this.max[d] >= other.max[d]) == false) {
+          // not contains:
+          return false;
+        }
+      }
+      return true;
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder b = new StringBuilder();
+      b.append("Box(");
+      b.append(min[0]);
+      b.append(" TO ");
+      b.append(max[0]);
+      for (int d=1; d<min.length; ++d) {
+        b.append(", ");
+        b.append(min[d]);
+        b.append(" TO ");
+        b.append(max[d]);
+      }
+      b.append(")");
+
+      return b.toString();
+    }
+  }
+}


[27/43] lucene-solr:feature/autoscaling: remove stale comment

Posted by sh...@apache.org.
remove stale comment


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

Branch: refs/heads/feature/autoscaling
Commit: a3f4896359bd0a113eacb0756ec2afe6c8d5d7b9
Parents: 0fb386a
Author: Mike McCandless <mi...@apache.org>
Authored: Sat Mar 11 06:41:49 2017 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Sat Mar 11 06:41:49 2017 -0500

----------------------------------------------------------------------
 lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a3f48963/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java b/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java
index 2ea0d0e..b1f507a 100644
--- a/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java
@@ -183,7 +183,6 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
 
   private BooleanQuery rewriteNoScoring() {
     BooleanQuery.Builder newQuery = new BooleanQuery.Builder();
-    // ignore disableCoord, which only matters for scores
     newQuery.setMinimumNumberShouldMatch(getMinimumNumberShouldMatch());
     for (BooleanClause clause : clauses) {
       if (clause.getOccur() == Occur.MUST) {


[16/43] lucene-solr:feature/autoscaling: SOLR-10244: TestCoreDiscovery fails if you run it as root.

Posted by sh...@apache.org.
SOLR-10244: TestCoreDiscovery fails if you run it as root.


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

Branch: refs/heads/feature/autoscaling
Commit: 6a6e30329843a86de1063a2c8f324eb3f9dbfd91
Parents: 8a54929
Author: Mark Miller <ma...@gmail.com>
Authored: Wed Mar 8 10:23:21 2017 -0500
Committer: Mark Miller <ma...@gmail.com>
Committed: Wed Mar 8 10:23:46 2017 -0500

----------------------------------------------------------------------
 solr/core/src/test/org/apache/solr/core/TestCoreDiscovery.java | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6a6e3032/solr/core/src/test/org/apache/solr/core/TestCoreDiscovery.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/TestCoreDiscovery.java b/solr/core/src/test/org/apache/solr/core/TestCoreDiscovery.java
index fa07de8..65d459a 100644
--- a/solr/core/src/test/org/apache/solr/core/TestCoreDiscovery.java
+++ b/solr/core/src/test/org/apache/solr/core/TestCoreDiscovery.java
@@ -337,6 +337,7 @@ public class TestCoreDiscovery extends SolrTestCaseJ4 {
 
     File toSet = new File(coreDir, "core1");
     assumeTrue("Cannot make " + toSet + " non-readable. Test aborted.", toSet.setReadable(false, false));
+    assumeFalse("Appears we are a super user, skip test", toSet.canRead());
     CoreContainer cc = init();
     try (SolrCore core1 = cc.getCore("core1");
          SolrCore core2 = cc.getCore("core2")) {
@@ -362,6 +363,7 @@ public class TestCoreDiscovery extends SolrTestCaseJ4 {
     File toSet = new File(solrHomeDirectory, "cantReadDir");
     assertTrue("Should have been able to make directory '" + toSet.getAbsolutePath() + "' ", toSet.mkdirs());
     assumeTrue("Cannot make " + toSet + " non-readable. Test aborted.", toSet.setReadable(false, false));
+    assumeFalse("Appears we are a super user, skip test", toSet.canRead());
     CoreContainer cc = init();
     try (SolrCore core1 = cc.getCore("core1");
          SolrCore core2 = cc.getCore("core2")) {
@@ -421,7 +423,7 @@ public class TestCoreDiscovery extends SolrTestCaseJ4 {
         new File(homeDir, "core1" + File.separator + CorePropertiesLocator.PROPERTIES_FILENAME));
 
     assumeTrue("Cannot make " + homeDir + " non-readable. Test aborted.", homeDir.setReadable(false, false));
-
+    assumeFalse("Appears we are a super user, skip test", homeDir.canRead());
     CoreContainer cc = null;
     try {
       cc = init();


[30/43] lucene-solr:feature/autoscaling: SOLR-10039: New LatLonPointSpatialField

Posted by sh...@apache.org.
SOLR-10039: New LatLonPointSpatialField


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

Branch: refs/heads/feature/autoscaling
Commit: 182c20c4e55c39362f6d46d388eb2b8e0a9052e6
Parents: 1745b03
Author: David Smiley <ds...@apache.org>
Authored: Sat Mar 11 20:48:01 2017 -0500
Committer: David Smiley <ds...@apache.org>
Committed: Sat Mar 11 20:48:01 2017 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../solr/schema/AbstractSpatialFieldType.java   |   2 +-
 .../solr/schema/LatLonPointSpatialField.java    | 272 +++++++++++++++++++
 .../solr/collection1/conf/schema-spatial.xml    |   4 +
 .../apache/solr/search/TestSolr4Spatial.java    |  61 ++++-
 .../basic_configs/conf/managed-schema           |   7 +-
 .../conf/managed-schema                         |   7 +-
 .../conf/managed-schema                         |   7 +-
 8 files changed, 338 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/182c20c4/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index b164405..7bf679f 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -166,6 +166,9 @@ New Features
 
 * SOLR-8045: Deploy V2 API at /v2 instead of /solr/v2 (Cao Manh Dat, Noble Paul)
 
+* SOLR-10039: New LatLonPointSpatialField replacement for LatLonType (and some uses of RPT).  Multi-value capable
+  indexed geo lat-lon points, query by rect or circle.  Efficient distance sorting/boosting too. (David Smiley)
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/182c20c4/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java b/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java
index 4e2829e..2106205 100644
--- a/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java
+++ b/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java
@@ -225,7 +225,7 @@ public abstract class AbstractSpatialFieldType<T extends SpatialStrategy> extend
     }
 
     List<IndexableField> result = new ArrayList<>();
-    if (field.indexed()) {
+    if (field.indexed() || field.hasDocValues()) {
       T strategy = getStrategy(field.getName());
       result.addAll(Arrays.asList(strategy.createIndexableFields(shape)));
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/182c20c4/solr/core/src/java/org/apache/solr/schema/LatLonPointSpatialField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/LatLonPointSpatialField.java b/solr/core/src/java/org/apache/solr/schema/LatLonPointSpatialField.java
new file mode 100644
index 0000000..c09856a
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/schema/LatLonPointSpatialField.java
@@ -0,0 +1,272 @@
+/*
+ * 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.schema;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.Objects;
+
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.LatLonDocValuesField;
+import org.apache.lucene.document.LatLonPoint;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
+import org.apache.lucene.search.FieldComparator;
+import org.apache.lucene.search.IndexOrDocValuesQuery;
+import org.apache.lucene.search.LeafFieldComparator;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.spatial.SpatialStrategy;
+import org.apache.lucene.spatial.query.SpatialArgs;
+import org.apache.lucene.spatial.query.SpatialOperation;
+import org.apache.lucene.spatial.query.UnsupportedSpatialOperation;
+import org.apache.solr.common.SolrException;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.distance.DistanceUtils;
+import org.locationtech.spatial4j.shape.Circle;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Rectangle;
+import org.locationtech.spatial4j.shape.Shape;
+
+/**
+ * A spatial implementation based on Lucene's {@code LatLonPoint} and {@code LatLonDocValuesField}. The
+ * first is based on Lucene's "Points" API, which is a BKD Index.  This field type is strictly limited to
+ * coordinates in lat/lon decimal degrees.  The accuracy is about a centimeter.
+ */
+// TODO once LLP & LLDVF are out of Lucene Sandbox, we should be able to javadoc reference them.
+public class LatLonPointSpatialField extends AbstractSpatialFieldType implements SchemaAware {
+  private IndexSchema schema;
+
+  // TODO handle polygons
+
+  @Override
+  public void checkSchemaField(SchemaField field) {
+    // override because if we didn't, FieldType will complain about docValues not being supported (we do support it)
+  }
+
+  @Override
+  public void inform(IndexSchema schema) {
+    this.schema = schema;
+  }
+
+  @Override
+  protected SpatialStrategy newSpatialStrategy(String fieldName) {
+    SchemaField schemaField = schema.getField(fieldName); // TODO change AbstractSpatialFieldType so we get schemaField?
+    return new LatLonPointSpatialStrategy(ctx, fieldName, schemaField.indexed(), schemaField.hasDocValues());
+  }
+
+  // TODO move to Lucene-spatial-extras once LatLonPoint & LatLonDocValuesField moves out of sandbox
+  public static class LatLonPointSpatialStrategy extends SpatialStrategy {
+
+    private final boolean indexed; // for query/filter
+    private final boolean docValues; // for sort. Can be used to query/filter.
+
+    public LatLonPointSpatialStrategy(SpatialContext ctx, String fieldName, boolean indexed, boolean docValues) {
+      super(ctx, fieldName);
+      if (!ctx.isGeo()) {
+        throw new IllegalArgumentException("ctx must be geo=true: " + ctx);
+      }
+      this.indexed = indexed;
+      this.docValues = docValues;
+    }
+
+    @Override
+    public Field[] createIndexableFields(Shape shape) {
+      if (!(shape instanceof Point)) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+            getClass().getSimpleName() + " only supports indexing points; got: " + shape);
+      }
+      Point point = (Point) shape;
+
+      int fieldsLen = (indexed ? 1 : 0) + (docValues ? 1 : 0);
+      Field[] fields = new Field[fieldsLen];
+      int fieldsIdx = 0;
+      if (indexed) {
+        fields[fieldsIdx++] = new LatLonPoint(getFieldName(), point.getY(), point.getX());
+      }
+      if (docValues) {
+        fields[fieldsIdx++] = new LatLonDocValuesField(getFieldName(), point.getY(), point.getX());
+      }
+      return fields;
+    }
+
+    @Override
+    public Query makeQuery(SpatialArgs args) {
+      if (args.getOperation() != SpatialOperation.Intersects) {
+        throw new UnsupportedSpatialOperation(args.getOperation());
+      }
+      Shape shape = args.getShape();
+      if (indexed && docValues) {
+        return new IndexOrDocValuesQuery(makeQueryFromIndex(shape), makeQueryFromDocValues(shape));
+      } else if (indexed) {
+        return makeQueryFromIndex(shape);
+      } else if (docValues) {
+        return makeQueryFromDocValues(shape);
+      } else {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+            getFieldName() + " needs indexed (preferred) or docValues to support search");
+      }
+    }
+
+    // Uses LatLonPoint
+    protected Query makeQueryFromIndex(Shape shape) {
+      // note: latitude then longitude order for LLP's methods
+      if (shape instanceof Circle) {
+        Circle circle = (Circle) shape;
+        double radiusMeters = circle.getRadius() * DistanceUtils.DEG_TO_KM * 1000;
+        return LatLonPoint.newDistanceQuery(getFieldName(),
+            circle.getCenter().getY(), circle.getCenter().getX(),
+            radiusMeters);
+      } else if (shape instanceof Rectangle) {
+        Rectangle rect = (Rectangle) shape;
+        return LatLonPoint.newBoxQuery(getFieldName(),
+            rect.getMinY(), rect.getMaxY(), rect.getMinX(), rect.getMaxX());
+      } else if (shape instanceof Point) {
+        Point point = (Point) shape;
+        return LatLonPoint.newDistanceQuery(getFieldName(),
+            point.getY(), point.getX(), 0);
+      } else {
+        throw new UnsupportedOperationException("Shape " + shape.getClass() + " is not supported by " + getClass());
+      }
+//      } else if (shape instanceof LucenePolygonShape) {
+//        // TODO support multi-polygon
+//        Polygon poly = ((LucenePolygonShape)shape).lucenePolygon;
+//        return LatLonPoint.newPolygonQuery(getFieldName(), poly);
+    }
+
+    // Uses DocValuesField  (otherwise identical to above)
+    protected Query makeQueryFromDocValues(Shape shape) {
+      // note: latitude then longitude order for LLP's methods
+      if (shape instanceof Circle) {
+        Circle circle = (Circle) shape;
+        double radiusMeters = circle.getRadius() * DistanceUtils.DEG_TO_KM * 1000;
+        return LatLonDocValuesField.newDistanceQuery(getFieldName(),
+            circle.getCenter().getY(), circle.getCenter().getX(),
+            radiusMeters);
+      } else if (shape instanceof Rectangle) {
+        Rectangle rect = (Rectangle) shape;
+        return LatLonDocValuesField.newBoxQuery(getFieldName(),
+            rect.getMinY(), rect.getMaxY(), rect.getMinX(), rect.getMaxX());
+      } else if (shape instanceof Point) {
+        Point point = (Point) shape;
+        return LatLonDocValuesField.newDistanceQuery(getFieldName(),
+            point.getY(), point.getX(), 0);
+      } else {
+        throw new UnsupportedOperationException("Shape " + shape.getClass() + " is not supported by " + getClass());
+      }
+//      } else if (shape instanceof LucenePolygonShape) {
+//        // TODO support multi-polygon
+//        Polygon poly = ((LucenePolygonShape)shape).lucenePolygon;
+//        return LatLonPoint.newPolygonQuery(getFieldName(), poly);
+    }
+
+    @Override
+    public ValueSource makeDistanceValueSource(Point queryPoint, double multiplier) {
+      if (!docValues) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+            getFieldName() + " must have docValues enabled to support this feature");
+      }
+      // Internally, the distance from LatLonPointSortField/Comparator is in meters. So we must also go from meters to
+      //  degrees, which is what Lucene spatial-extras is oriented around.
+      return new DistanceSortValueSource(getFieldName(), queryPoint,
+          DistanceUtils.KM_TO_DEG / 1000.0 * multiplier);
+    }
+
+    /**
+     * A {@link ValueSource} based around {@code LatLonDocValuesField#newDistanceSort(String, double, double)}.
+     */
+    private static class DistanceSortValueSource extends ValueSource {
+      private final String fieldName;
+      private final Point queryPoint;
+      private final double multiplier;
+
+      DistanceSortValueSource(String fieldName, Point queryPoint, double multiplier) {
+        this.fieldName = fieldName;
+        this.queryPoint = queryPoint;
+        this.multiplier = multiplier;
+      }
+
+      @Override
+      public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        DistanceSortValueSource that = (DistanceSortValueSource) o;
+        return Double.compare(that.multiplier, multiplier) == 0 &&
+            Objects.equals(fieldName, that.fieldName) &&
+            Objects.equals(queryPoint, that.queryPoint);
+      }
+
+      @Override
+      public int hashCode() {
+        return Objects.hash(fieldName, queryPoint, multiplier);
+      }
+
+      @Override
+      public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
+        return new DoubleDocValues(this) {
+          @SuppressWarnings("unchecked")
+          final FieldComparator<Double> comparator =
+              (FieldComparator<Double>) getSortField(false).getComparator(1, 1);
+          final LeafFieldComparator leafComparator = comparator.getLeafComparator(readerContext);
+          final double mult = multiplier; // so it's a local field
+
+          // Since this computation is expensive, it's worth caching it just in case.
+          double cacheDoc = -1;
+          double cacheVal = Double.POSITIVE_INFINITY;
+
+          @Override
+          public double doubleVal(int doc) {
+            if (cacheDoc != doc) {
+              try {
+                leafComparator.copy(0, doc);
+                cacheVal = comparator.value(0) * mult;
+                cacheDoc = doc;
+              } catch (IOException e) {
+                throw new RuntimeException(e);
+              }
+            }
+            return cacheVal;
+          }
+
+          @Override
+          public boolean exists(int doc) {
+            return !Double.isInfinite(doubleVal(doc));
+          }
+        };
+      }
+
+      @Override
+      public String description() {
+        return "distSort(" + fieldName + ", " + queryPoint + ", mult:" + multiplier + ")";
+      }
+
+      @Override
+      public SortField getSortField(boolean reverse) {
+        if (reverse) {
+          return super.getSortField(true); // will use an impl that calls getValues
+        }
+        return LatLonDocValuesField.newDistanceSort(fieldName, queryPoint.getY(), queryPoint.getX());
+      }
+
+    }
+
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/182c20c4/solr/core/src/test-files/solr/collection1/conf/schema-spatial.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-spatial.xml b/solr/core/src/test-files/solr/collection1/conf/schema-spatial.xml
index 254f58b..9c7a36f 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema-spatial.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema-spatial.xml
@@ -54,6 +54,7 @@
   <fieldType name="bbox" class="solr.BBoxField"
              numberType="tdoubleDV" distanceUnits="degrees" storeSubFields="false"/>
 
+  <fieldType name="llp" class="solr.LatLonPointSpatialField" distanceUnits="degrees" multiValued="true" />
 
   <field name="id" type="string" required="true"/>
 
@@ -64,6 +65,9 @@
   <field name="pointvector" type="pointvector"/>
   <field name="srptgeom" type="srptgeom"/>
   <field name="bbox" type="bbox"/>
+  <field name="llp" type="llp" indexed="true" docValues="true" />
+  <field name="llp_idx" type="llp" indexed="true" docValues="false" />
+  <field name="llp_dv" type="llp" indexed="false" docValues="true" />
 
   <dynamicField name="bboxD_*" type="bbox" indexed="true"/>
   <dynamicField name="str_*" type="string" indexed="true" stored="true"/>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/182c20c4/solr/core/src/test/org/apache/solr/search/TestSolr4Spatial.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestSolr4Spatial.java b/solr/core/src/test/org/apache/solr/search/TestSolr4Spatial.java
index 2fe3740..8cd96ae 100644
--- a/solr/core/src/test/org/apache/solr/search/TestSolr4Spatial.java
+++ b/solr/core/src/test/org/apache/solr/search/TestSolr4Spatial.java
@@ -19,7 +19,6 @@ package org.apache.solr.search;
 import java.text.ParseException;
 import java.util.Arrays;
 
-import com.carrotsearch.randomizedtesting.RandomizedTest;
 import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
 import org.locationtech.spatial4j.context.SpatialContext;
 import org.locationtech.spatial4j.distance.DistanceUtils;
@@ -43,16 +42,18 @@ import org.junit.Test;
  */
 public class TestSolr4Spatial extends SolrTestCaseJ4 {
 
-  private String fieldName;
+  private final String fieldName;
+  private final boolean canCalcDistance;
 
   public TestSolr4Spatial(String fieldName) {
     this.fieldName = fieldName;
+    this.canCalcDistance = !fieldName.equals("llp_idx");
   }
 
   @ParametersFactory
   public static Iterable<Object[]> parameters() {
     return Arrays.asList(new Object[][]{
-        {"srpt_geohash"}, {"srpt_quad"}, {"srpt_packedquad"}, {"stqpt_geohash"}, {"pointvector"}, {"bbox"}
+        {"llp"}, {"llp_idx"}, {"llp_dv"}, {"srpt_geohash"}, {"srpt_quad"}, {"srpt_packedquad"}, {"stqpt_geohash"}, {"pointvector"}, {"bbox"}
     });
   }
 
@@ -105,6 +106,10 @@ public class TestSolr4Spatial extends SolrTestCaseJ4 {
     assertU(adoc("id", "11", fieldName, "89.9,-130"));
     assertU(adoc("id", "12", fieldName, "-89.9,50"));
     assertU(adoc("id", "13", fieldName, "-89.9,-130"));
+    if (random().nextBoolean()) {
+      assertU(commit());
+    }
+    assertU(adoc("id", "99"));//blank
     assertU(commit());
   }
 
@@ -192,7 +197,7 @@ public class TestSolr4Spatial extends SolrTestCaseJ4 {
     //Test using the Lucene spatial syntax
     {
       //never actually need the score but lets test
-      String score = new String[]{null, "none","distance","recipDistance"}[random().nextInt(4)];
+      String score = randomScoreMode();
 
       double distDEG = DistanceUtils.dist2Degrees(distKM, DistanceUtils.EARTH_MEAN_RADIUS_KM);
       Point point = SpatialUtils.parsePoint(ptStr, SpatialContext.GEO);
@@ -225,6 +230,10 @@ public class TestSolr4Spatial extends SolrTestCaseJ4 {
 
   }
 
+  private String randomScoreMode() {
+    return canCalcDistance ? new String[]{null, "none","distance","recipDistance"}[random().nextInt(4)] : "none";
+  }
+
   @Test
   public void testRangeSyntax() {
     setupDocs();
@@ -232,10 +241,10 @@ public class TestSolr4Spatial extends SolrTestCaseJ4 {
     int docId = 1;
     int count = 1;
 
-    String score = random().nextBoolean() ? "none" : "distance";//never actually need the score but lets test
+    String score = randomScoreMode();//never actually need the score but lets test
     assertQ(req(
         "fl", "id", "q","*:*", "rows", "1000",    // testing quotes in range too
-        "fq", "{! score="+score+" df="+fieldName+"}[32,-80 TO \"33 , -79\"]"),//lower-left to upper-right
+        "fq", "{! "+(score==null?"":" score="+score)+" df="+fieldName+"}[32,-80 TO \"33 , -79\"]"),//lower-left to upper-right
 
         "//result/doc/*[@name='id'][.='" + docId + "']",
         "*[count(//doc)=" + count + "]");
@@ -243,13 +252,46 @@ public class TestSolr4Spatial extends SolrTestCaseJ4 {
 
   @Test
   public void testSort() throws Exception {
+    assumeTrue("dist sorting not supported on field " + fieldName, canCalcDistance);
     assertU(adoc("id", "100", fieldName, "1,2"));
     assertU(adoc("id", "101", fieldName, "4,-1"));
-    assertU(adoc("id", "999", fieldName, "70,70"));//far away from these queries
+    if (random().nextBoolean()) {
+      assertU(commit()); // new segment
+    }
+    if (random().nextBoolean()) {
+      assertU(adoc("id", "999", fieldName, "70,70"));//far away from these queries; we filter it out
+    } else {
+      assertU(adoc("id", "999")); // no data
+    }
     assertU(commit());
 
-    //test absence of score=distance means it doesn't score
 
+    // geodist asc
+    assertJQ(req(
+        "q", radiusQuery(3, 4, 9, null, null),
+        "fl","id",
+        "sort","geodist() asc",
+        "sfield", fieldName, "pt", "3,4")
+        , 1e-3
+        , "/response/docs/[0]/id=='100'"
+        , "/response/docs/[1]/id=='101'"
+    );
+    // geodist desc  (simply reverse the assertions)
+    assertJQ(req(
+        "q", radiusQuery(3, 4, 9, null, null),
+        "fl","id",
+        "sort","geodist() desc", // DESC
+        "sfield", fieldName, "pt", "3,4")
+        , 1e-3
+        , "/response/docs/[0]/id=='101'" // FLIPPED
+        , "/response/docs/[1]/id=='100'" // FLIPPED
+    );
+
+    //
+    //  NOTE: the rest work via the score of the spatial query. Generally, you should use geodist() instead.
+    //
+
+    //test absence of score=distance means it doesn't score
     assertJQ(req(
         "q", radiusQuery(3, 4, 9, null, null),
         "fl","id,score")
@@ -345,7 +387,8 @@ public class TestSolr4Spatial extends SolrTestCaseJ4 {
 
   @Test
   public void testSortMultiVal() throws Exception {
-    RandomizedTest.assumeFalse("Multivalue not supported for this field",
+    assumeTrue("dist sorting not supported on field " + fieldName, canCalcDistance);
+    assumeFalse("Multivalue not supported for this field",
         fieldName.equals("pointvector") || fieldName.equals("bbox"));
 
     assertU(adoc("id", "100", fieldName, "1,2"));//1 point

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/182c20c4/solr/server/solr/configsets/basic_configs/conf/managed-schema
----------------------------------------------------------------------
diff --git a/solr/server/solr/configsets/basic_configs/conf/managed-schema b/solr/server/solr/configsets/basic_configs/conf/managed-schema
index 60a0e98..22553d8 100644
--- a/solr/server/solr/configsets/basic_configs/conf/managed-schema
+++ b/solr/server/solr/configsets/basic_configs/conf/managed-schema
@@ -148,9 +148,6 @@
     <dynamicField name="*_d"  type="double" indexed="true"  stored="true"/>
     <dynamicField name="*_ds" type="doubles" indexed="true"  stored="true"/>
 
-    <!-- Type used to index the lat and lon components for the "location" FieldType -->
-    <dynamicField name="*_coordinate"  type="tdouble" indexed="true"  stored="false" useDocValuesAsStored="false" />
-
     <dynamicField name="*_dt"  type="date"    indexed="true"  stored="true"/>
     <dynamicField name="*_dts" type="date"    indexed="true"  stored="true" multiValued="true"/>
     <dynamicField name="*_p"  type="location" indexed="true" stored="true"/>
@@ -551,8 +548,8 @@
     <dynamicField name="*_point" type="point"  indexed="true"  stored="true"/>
     <fieldType name="point" class="solr.PointType" dimension="2" subFieldSuffix="_d"/>
 
-    <!-- A specialized field for geospatial search. If indexed, this fieldType must not be multivalued. -->
-    <fieldType name="location" class="solr.LatLonType" subFieldSuffix="_coordinate"/>
+    <!-- A specialized field for geospatial search filters and distance sorting. -->
+    <fieldType name="location" class="solr.LatLonPointSpatialField" docValues="true"/>
 
     <!-- An alternative geospatial field type new to Solr 4.  It supports multiValued and polygon shapes.
       For more information about this and other Spatial fields new to Solr 4, see:

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/182c20c4/solr/server/solr/configsets/data_driven_schema_configs/conf/managed-schema
----------------------------------------------------------------------
diff --git a/solr/server/solr/configsets/data_driven_schema_configs/conf/managed-schema b/solr/server/solr/configsets/data_driven_schema_configs/conf/managed-schema
index b1373d8..558c05e 100644
--- a/solr/server/solr/configsets/data_driven_schema_configs/conf/managed-schema
+++ b/solr/server/solr/configsets/data_driven_schema_configs/conf/managed-schema
@@ -148,9 +148,6 @@
     <dynamicField name="*_d"  type="double" indexed="true"  stored="true"/>
     <dynamicField name="*_ds" type="doubles" indexed="true"  stored="true"/>
 
-    <!-- Type used to index the lat and lon components for the "location" FieldType -->
-    <dynamicField name="*_coordinate"  type="tdouble" indexed="true"  stored="false" useDocValuesAsStored="false" />
-
     <dynamicField name="*_dt"  type="date"    indexed="true"  stored="true"/>
     <dynamicField name="*_dts" type="date"    indexed="true"  stored="true" multiValued="true"/>
     <dynamicField name="*_p"  type="location" indexed="true" stored="true"/>
@@ -551,8 +548,8 @@
     <dynamicField name="*_point" type="point"  indexed="true"  stored="true"/>
     <fieldType name="point" class="solr.PointType" dimension="2" subFieldSuffix="_d"/>
 
-    <!-- A specialized field for geospatial search. If indexed, this fieldType must not be multivalued. -->
-    <fieldType name="location" class="solr.LatLonType" subFieldSuffix="_coordinate"/>
+    <!-- A specialized field for geospatial search filters and distance sorting. -->
+    <fieldType name="location" class="solr.LatLonPointSpatialField" docValues="true"/>
 
     <!-- An alternative geospatial field type new to Solr 4.  It supports multiValued and polygon shapes.
       For more information about this and other Spatial fields new to Solr 4, see:

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/182c20c4/solr/server/solr/configsets/sample_techproducts_configs/conf/managed-schema
----------------------------------------------------------------------
diff --git a/solr/server/solr/configsets/sample_techproducts_configs/conf/managed-schema b/solr/server/solr/configsets/sample_techproducts_configs/conf/managed-schema
index 4980540..bd292a0 100644
--- a/solr/server/solr/configsets/sample_techproducts_configs/conf/managed-schema
+++ b/solr/server/solr/configsets/sample_techproducts_configs/conf/managed-schema
@@ -212,9 +212,6 @@
    <dynamicField name="*_d"  type="double" indexed="true"  stored="true"/>
    <dynamicField name="*_ds" type="double" indexed="true"  stored="true"  multiValued="true"/>
 
-   <!-- Type used to index the lat and lon components for the "location" FieldType -->
-   <dynamicField name="*_coordinate"  type="tdouble" indexed="true"  stored="false" useDocValuesAsStored="false" />
-
    <dynamicField name="*_dt"  type="date"    indexed="true"  stored="true"/>
    <dynamicField name="*_dts" type="date"    indexed="true"  stored="true" multiValued="true"/>
    <dynamicField name="*_p"  type="location" indexed="true" stored="true"/>
@@ -696,8 +693,8 @@
      -->
     <fieldType name="point" class="solr.PointType" dimension="2" subFieldSuffix="_d"/>
 
-    <!-- A specialized field for geospatial search. If indexed, this fieldType must not be multivalued. -->
-    <fieldType name="location" class="solr.LatLonType" subFieldSuffix="_coordinate"/>
+    <!-- A specialized field for geospatial search filters and distance sorting. -->
+    <fieldType name="location" class="solr.LatLonPointSpatialField" docValues="true"/>
 
     <!-- An alternative geospatial field type new to Solr 4.  It supports multiValued and polygon shapes.
       For more information about this and other Spatial fields new to Solr 4, see:


[38/43] lucene-solr:feature/autoscaling: SOLR-9838: 'inc' atomic update doesn't respect default field value

Posted by sh...@apache.org.
SOLR-9838: 'inc' atomic update doesn't respect default field value


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

Branch: refs/heads/feature/autoscaling
Commit: a06c39f3e50a1616cd7c48f4454dd77be17c7278
Parents: d5181ec
Author: Ishan Chattopadhyaya <is...@apache.org>
Authored: Mon Mar 13 18:46:08 2017 +0530
Committer: Ishan Chattopadhyaya <is...@apache.org>
Committed: Mon Mar 13 18:46:08 2017 +0530

----------------------------------------------------------------------
 solr/CHANGES.txt                                         |  2 ++
 .../update/processor/AtomicUpdateDocumentMerger.java     | 11 ++++++-----
 .../apache/solr/update/processor/AtomicUpdatesTest.java  |  3 +--
 3 files changed, 9 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a06c39f3/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 7bf679f..6e96cbb 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -227,6 +227,8 @@ Bug Fixes
 
 * SOLR-10226: add back "totalTime" metric to all handlers. See also the back-compat note. (ab)
 
+* SOLR-9838: "inc" atomic update doesn't respect default field value (hoss, Amrit Sarkar, Ishan Chattopadhyaya)
+
 Optimizations
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a06c39f3/solr/core/src/java/org/apache/solr/update/processor/AtomicUpdateDocumentMerger.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/processor/AtomicUpdateDocumentMerger.java b/solr/core/src/java/org/apache/solr/update/processor/AtomicUpdateDocumentMerger.java
index 093149a..9061235 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/AtomicUpdateDocumentMerger.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/AtomicUpdateDocumentMerger.java
@@ -316,12 +316,11 @@ public class AtomicUpdateDocumentMerger {
 
   protected void doInc(SolrInputDocument toDoc, SolrInputField sif, Object fieldVal) {
     SolrInputField numericField = toDoc.get(sif.getName());
-    if (numericField == null) {
-      toDoc.setField(sif.getName(),  fieldVal);
-    } else {
+    SchemaField sf = schema.getField(sif.getName());
+    if (numericField != null || sf.getDefaultValue() != null) {
       // TODO: fieldtype needs externalToObject?
-      String oldValS = numericField.getFirstValue().toString();
-      SchemaField sf = schema.getField(sif.getName());
+      String oldValS = (numericField != null) ?
+          numericField.getFirstValue().toString(): sf.getDefaultValue().toString();
       BytesRefBuilder term = new BytesRefBuilder();
       sf.getType().readableToIndexed(oldValS, term);
       Object oldVal = sf.getType().toObject(sf, term.get());
@@ -340,6 +339,8 @@ public class AtomicUpdateDocumentMerger {
       }
 
       toDoc.setField(sif.getName(),  result);
+    } else {
+      toDoc.setField(sif.getName(), fieldVal);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a06c39f3/solr/core/src/test/org/apache/solr/update/processor/AtomicUpdatesTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/processor/AtomicUpdatesTest.java b/solr/core/src/test/org/apache/solr/update/processor/AtomicUpdatesTest.java
index 7bae2c9..bfcf015 100644
--- a/solr/core/src/test/org/apache/solr/update/processor/AtomicUpdatesTest.java
+++ b/solr/core/src/test/org/apache/solr/update/processor/AtomicUpdatesTest.java
@@ -1204,7 +1204,6 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
     
   }
 
-  @AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/SOLR-9838")
   public void testAtomicUpdateOfFieldsWithDefaultValue() {
     // both fields have the same default value (42)
     for (String fieldToUpdate : Arrays.asList("intDefault", "intDvoDefault")) {
@@ -1254,7 +1253,7 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
               , "count(//doc/*)=6"
               );
       // do atomic update
-      assertU(adoc(sdoc("id", "7", fieldToUpdate, ImmutableMap.of("inc", -555))));
+      assertU(adoc(sdoc("id", "8", fieldToUpdate, ImmutableMap.of("inc", -555))));
       assertQ(fieldToUpdate + ": RTG after atomic update"
               , req("qt", "/get", "id", "8")
               , "count(//doc)=1"


[36/43] lucene-solr:feature/autoscaling: SOLR-10079: Force in-place standalone test to always use NoMergePolicy, also assert that it was used

Posted by sh...@apache.org.
SOLR-10079: Force in-place standalone test to always use NoMergePolicy, also assert that it was used


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

Branch: refs/heads/feature/autoscaling
Commit: b64382bb07ebae4c6f8711b5e0fb4341d2f09f4e
Parents: 35e0c05
Author: Ishan Chattopadhyaya <is...@apache.org>
Authored: Mon Mar 13 15:22:49 2017 +0530
Committer: Ishan Chattopadhyaya <is...@apache.org>
Committed: Mon Mar 13 15:22:49 2017 +0530

----------------------------------------------------------------------
 .../update/TestInPlaceUpdatesStandalone.java    | 27 ++++++++++++++++++++
 1 file changed, 27 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b64382bb/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesStandalone.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesStandalone.java b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesStandalone.java
index 9a5031f..877467e 100644
--- a/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesStandalone.java
+++ b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesStandalone.java
@@ -32,6 +32,8 @@ import java.util.Random;
 import java.util.Set;
 
 import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.NoMergePolicy;
 import org.apache.lucene.util.TestUtil;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.client.solrj.SolrClient;
@@ -41,6 +43,7 @@ import org.apache.solr.common.SolrDocumentList;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.SolrInputField;
+import org.apache.solr.index.NoMergePolicyFactory;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.update.processor.DistributedUpdateProcessor;
 import org.apache.solr.schema.IndexSchema;
@@ -49,6 +52,7 @@ import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.update.processor.AtomicUpdateDocumentMerger;
 import org.apache.solr.util.RefCounted;
 import org.junit.After;
+import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -67,6 +71,14 @@ public class TestInPlaceUpdatesStandalone extends SolrTestCaseJ4 {
     System.setProperty("solr.tests.floatClassName", random().nextBoolean()? "TrieFloatField": "FloatPointField");
     System.setProperty("solr.tests.doubleClassName", random().nextBoolean()? "TrieDoubleField": "DoublePointField");
 
+    // we need consistent segments that aren't re-ordered on merge because we're
+    // asserting inplace updates happen by checking the internal [docid]
+    systemSetPropertySolrTestsMergePolicyFactory(NoMergePolicyFactory.class.getName());
+
+    // HACK: Don't use a RandomMergePolicy, but only use the mergePolicyFactory that we've just set
+    System.setProperty(SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICYFACTORY, "true");
+    System.setProperty(SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICY, "false");
+
     initCore("solrconfig-tlog.xml", "schema-inplace-updates.xml");
 
     // sanity check that autocommits are disabled
@@ -75,6 +87,16 @@ public class TestInPlaceUpdatesStandalone extends SolrTestCaseJ4 {
     assertEquals(-1, h.getCore().getSolrConfig().getUpdateHandlerInfo().autoCommmitMaxDocs);
     assertEquals(-1, h.getCore().getSolrConfig().getUpdateHandlerInfo().autoSoftCommmitMaxDocs);
 
+    // assert that NoMergePolicy was chosen
+    RefCounted<IndexWriter> iw = h.getCore().getSolrCoreState().getIndexWriter(h.getCore());
+    try {
+      IndexWriter writer = iw.get();
+      assertTrue("Actual merge policy is: " + writer.getConfig().getMergePolicy(),
+          writer.getConfig().getMergePolicy() instanceof NoMergePolicy); 
+    } finally {
+      iw.decref();
+    }
+
     // validate that the schema was not changed to an unexpected state
     IndexSchema schema = h.getCore().getLatestSchema();
     for (String fieldName : Arrays.asList("_version_",
@@ -98,6 +120,11 @@ public class TestInPlaceUpdatesStandalone extends SolrTestCaseJ4 {
     client = new EmbeddedSolrServer(h.getCoreContainer(), h.coreName);
   }
 
+  @AfterClass
+  public static void afterClass() {
+    client = null;
+  }
+
   @After
   public void after() {
     System.clearProperty("solr.tests.intClassName");


[10/43] lucene-solr:feature/autoscaling: Add 6.4.2 back compat test indexes

Posted by sh...@apache.org.
Add 6.4.2 back compat test indexes


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

Branch: refs/heads/feature/autoscaling
Commit: 3a9933960ec35e3083f261549dfed0e75fd8268c
Parents: 57e8543
Author: Ishan Chattopadhyaya <is...@apache.org>
Authored: Wed Mar 8 01:21:45 2017 +0530
Committer: Ishan Chattopadhyaya <is...@apache.org>
Committed: Wed Mar 8 01:21:45 2017 +0530

----------------------------------------------------------------------
 .../lucene/index/TestBackwardsCompatibility.java   |   4 +++-
 .../org/apache/lucene/index/index.6.4.2-cfs.zip    | Bin 0 -> 15856 bytes
 .../org/apache/lucene/index/index.6.4.2-nocfs.zip  | Bin 0 -> 15886 bytes
 3 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a993396/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
index 57ce52a..1dda6b6 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
@@ -295,7 +295,9 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
     "6.4.0-cfs",
     "6.4.0-nocfs",
     "6.4.1-cfs",
-    "6.4.1-nocfs"
+    "6.4.1-nocfs",
+    "6.4.2-cfs",
+    "6.4.2-nocfs"
   };
   
   final String[] unsupportedNames = {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a993396/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.4.2-cfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.4.2-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.4.2-cfs.zip
new file mode 100644
index 0000000..eee89f4
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.4.2-cfs.zip differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a993396/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.4.2-nocfs.zip
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.4.2-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.4.2-nocfs.zip
new file mode 100644
index 0000000..d55a6f6
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.4.2-nocfs.zip differ


[39/43] lucene-solr:feature/autoscaling: SOLR-10250: CloudSolrClient can now return versions for documents added or deleted when versions=true is passed

Posted by sh...@apache.org.
SOLR-10250: CloudSolrClient can now return versions for documents added or deleted when versions=true is passed


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

Branch: refs/heads/feature/autoscaling
Commit: ceffbf98445065220dcc9380e2731fd26a467f5d
Parents: a06c39f
Author: Ishan Chattopadhyaya <is...@apache.org>
Authored: Mon Mar 13 19:53:22 2017 +0530
Committer: Ishan Chattopadhyaya <is...@apache.org>
Committed: Mon Mar 13 19:53:22 2017 +0530

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  4 ++
 .../solr/client/solrj/impl/CloudSolrClient.java | 17 ++++++-
 .../client/solrj/impl/CloudSolrClientTest.java  | 48 ++++++++++++++++++++
 3 files changed, 68 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ceffbf98/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 6e96cbb..5d6d9d7 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -169,6 +169,10 @@ New Features
 * SOLR-10039: New LatLonPointSpatialField replacement for LatLonType (and some uses of RPT).  Multi-value capable
   indexed geo lat-lon points, query by rect or circle.  Efficient distance sorting/boosting too. (David Smiley)
 
+* SOLR-10250: CloudSolrClient can now return versions for documents added or deleted when "versions=true" is passed.
+  However, if there is a leader election while this request is in transit, the versions may not be returned from that
+  shard. (Boris Naguet, Ishan Chattopadhyaya)
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ceffbf98/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
index d3938c8..6941a77 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
@@ -903,7 +903,10 @@ public class CloudSolrClient extends SolrClient {
     // TolerantUpdateProcessor
     List<SimpleOrderedMap<String>> toleratedErrors = null; 
     int maxToleratedErrors = Integer.MAX_VALUE;
-      
+
+    // For "adds", "deletes", "deleteByQuery" etc.
+    Map<String, NamedList> versions = new HashMap<>();
+
     for(int i=0; i<response.size(); i++) {
       NamedList shardResponse = (NamedList)response.getVal(i);
       NamedList header = (NamedList)shardResponse.get("responseHeader");      
@@ -937,6 +940,15 @@ public class CloudSolrClient extends SolrClient {
           toleratedErrors.add(err);
         }
       }
+      for (String updateType: Arrays.asList("adds", "deletes", "deleteByQuery")) {
+        Object obj = shardResponse.get(updateType);
+        if (obj instanceof NamedList) {
+          NamedList versionsList = versions.containsKey(updateType) ?
+              versions.get(updateType): new NamedList();
+          versionsList.addAll((NamedList)obj);
+          versions.put(updateType, versionsList);
+        }
+      }
     }
 
     NamedList cheader = new NamedList();
@@ -971,6 +983,9 @@ public class CloudSolrClient extends SolrClient {
         throw toThrow;
       }
     }
+    for (String updateType: versions.keySet()) {
+      condensed.add(updateType, versions.get(updateType));
+    }
     condensed.add("responseHeader", cheader);
     return condensed;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ceffbf98/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java
index dd0dd16..d22b37c 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java
@@ -49,6 +49,7 @@ import org.apache.solr.client.solrj.response.RequestStatusState;
 import org.apache.solr.client.solrj.response.UpdateResponse;
 import org.apache.solr.cloud.AbstractDistribZkTestBase;
 import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrDocumentList;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
@@ -61,6 +62,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.ShardParams;
+import org.apache.solr.common.params.UpdateParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.handler.admin.CollectionsHandler;
@@ -633,6 +635,52 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
     }
   }
 
+  @Test
+  public void testVersionsAreReturned() throws Exception {
+    
+    // assert that "adds" are returned
+    UpdateRequest updateRequest = new UpdateRequest()
+        .add("id", "1", "a_t", "hello1")
+        .add("id", "2", "a_t", "hello2");
+    updateRequest.setParam(UpdateParams.VERSIONS, Boolean.TRUE.toString());
+
+    NamedList<Object> response = updateRequest.commit(cluster.getSolrClient(), COLLECTION).getResponse();
+    Object addsObject = response.get("adds");
+    
+    assertNotNull("There must be a adds parameter", addsObject);
+    assertTrue(addsObject instanceof NamedList<?>);
+    NamedList<?> adds = (NamedList<?>) addsObject;
+    assertEquals("There must be 2 versions (one per doc)", 2, adds.size());
+
+    Map<String, Long> versions = new HashMap<>();
+    Object object = adds.get("1");
+    assertNotNull("There must be a version for id 1", object);
+    assertTrue("Version for id 1 must be a long", object instanceof Long);
+    versions.put("1", (Long) object);
+
+    object = adds.get("2");
+    assertNotNull("There must be a version for id 2", object);
+    assertTrue("Version for id 2 must be a long", object instanceof Long);
+    versions.put("2", (Long) object);
+
+    QueryResponse resp = cluster.getSolrClient().query(COLLECTION, new SolrQuery("*:*"));
+    assertEquals("There should be one document because overwrite=true", 2, resp.getResults().getNumFound());
+
+    for (SolrDocument doc : resp.getResults()) {
+      Long version = versions.get(doc.getFieldValue("id"));
+      assertEquals("Version on add must match _version_ field", version, doc.getFieldValue("_version_"));
+    }
+
+    // assert that "deletes" are returned
+    UpdateRequest deleteRequest = new UpdateRequest().deleteById("1");
+    deleteRequest.setParam(UpdateParams.VERSIONS, Boolean.TRUE.toString());
+    response = deleteRequest.commit(cluster.getSolrClient(), COLLECTION).getResponse();
+    Object deletesObject = response.get("deletes");
+    assertNotNull("There must be a deletes parameter", deletesObject);
+    NamedList deletes = (NamedList) deletesObject;
+    assertEquals("There must be 1 version", 1, deletes.size());
+  }
+  
   private static void checkSingleServer(NamedList<Object> response) {
     final CloudSolrClient.RouteResponse rr = (CloudSolrClient.RouteResponse) response;
     final Map<String,LBHttpSolrClient.Req> routes = rr.getRoutes();


[25/43] lucene-solr:feature/autoscaling: LUCENE-7734: move to 7x section; won't do 6x backport

Posted by sh...@apache.org.
LUCENE-7734: move to 7x section; won't do 6x backport


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

Branch: refs/heads/feature/autoscaling
Commit: 6415d912ca370c47ac9bd138d719b0ade71893a1
Parents: 9540bc3
Author: David Smiley <ds...@apache.org>
Authored: Fri Mar 10 21:35:53 2017 -0500
Committer: David Smiley <ds...@apache.org>
Committed: Fri Mar 10 21:35:53 2017 -0500

----------------------------------------------------------------------
 lucene/CHANGES.txt | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6415d912/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index b6ee4b8..9407dfa 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -41,6 +41,9 @@ API Changes
   index-time scoring factors should be indexed into a doc value field and
   combined at query time using eg. FunctionScoreQuery. (Adrien Grand)
 
+* LUCENE-7734: FieldType's copy constructor was widened to accept any IndexableFieldType.
+  (David Smiley)
+
 Bug Fixes
 
 * LUCENE-7626: IndexWriter will no longer accept broken token offsets


[19/43] lucene-solr:feature/autoscaling: SOLR-10254: significantTerms Streaming Expression should work in non-SolrCloud mode

Posted by sh...@apache.org.
SOLR-10254: significantTerms Streaming Expression should work in non-SolrCloud mode


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

Branch: refs/heads/feature/autoscaling
Commit: 682c6a7d5145129e8ae01ff00505ddf5a564d396
Parents: 8756be0
Author: Joel Bernstein <jb...@apache.org>
Authored: Wed Mar 8 21:10:56 2017 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Wed Mar 8 21:11:26 2017 -0500

----------------------------------------------------------------------
 .../org/apache/solr/handler/StreamHandler.java  |  27 +++
 .../solrj/io/stream/SignificantTermsStream.java |  49 +---
 .../client/solrj/io/stream/TupleStream.java     |  94 ++++++++
 .../solrj/io/stream/StreamExpressionTest.java   | 234 +++++++++++++------
 4 files changed, 285 insertions(+), 119 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/682c6a7d/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
index 31b37e7..06e59b6 100644
--- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
@@ -18,6 +18,7 @@ package org.apache.solr.handler;
 
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
@@ -246,6 +247,7 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
     int worker = params.getInt("workerID", 0);
     int numWorkers = params.getInt("numWorkers", 1);
     StreamContext context = new StreamContext();
+    context.put("shards", getCollectionShards(params));
     context.workerID = worker;
     context.numWorkers = numWorkers;
     context.setSolrClientCache(clientCache);
@@ -509,4 +511,29 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
       return tuple;
     }
   }
+
+  private Map<String, List<String>> getCollectionShards(SolrParams params) {
+
+    Map<String, List<String>> collectionShards = new HashMap();
+    Iterator<String> paramsIt = params.getParameterNamesIterator();
+    while(paramsIt.hasNext()) {
+      String param = paramsIt.next();
+      if(param.indexOf(".shards") > -1) {
+        String collection = param.split("\\.")[0];
+        String shardString = params.get(param);
+        String[] shards = shardString.split(",");
+        List<String> shardList = new ArrayList();
+        for(String shard : shards) {
+          shardList.add(shard);
+        }
+        collectionShards.put(collection, shardList);
+      }
+    }
+
+    if(collectionShards.size() > 0) {
+      return collectionShards;
+    } else {
+      return null;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/682c6a7d/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SignificantTermsStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SignificantTermsStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SignificantTermsStream.java
index 87b5a9f..2acee51 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SignificantTermsStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SignificantTermsStream.java
@@ -74,12 +74,9 @@ public class SignificantTermsStream extends TupleStream implements Expressible{
 
   protected transient SolrClientCache cache;
   protected transient boolean isCloseCache;
-  protected transient CloudSolrClient cloudSolrClient;
-
   protected transient StreamContext streamContext;
   protected ExecutorService executorService;
 
-
   public SignificantTermsStream(String zkHost,
                                  String collectionName,
                                  Map params,
@@ -168,12 +165,12 @@ public class SignificantTermsStream extends TupleStream implements Expressible{
     String zkHost = null;
     if(null == zkHostExpression){
       zkHost = factory.getCollectionZkHost(collectionName);
-    }
-    else if(zkHostExpression.getParameter() instanceof StreamExpressionValue){
+    } else if(zkHostExpression.getParameter() instanceof StreamExpressionValue) {
       zkHost = ((StreamExpressionValue)zkHostExpression.getParameter()).getValue();
     }
-    if(null == zkHost){
-      throw new IOException(String.format(Locale.ROOT,"invalid expression %s - zkHost not found for collection '%s'",expression,collectionName));
+
+    if(zkHost == null){
+      zkHost = factory.getDefaultZkHost();
     }
 
     // We've got all the required items
@@ -238,47 +235,13 @@ public class SignificantTermsStream extends TupleStream implements Expressible{
       isCloseCache = false;
     }
 
-    this.cloudSolrClient = this.cache.getCloudSolrClient(zkHost);
-    this.executorService = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("FeaturesSelectionStream"));
+    this.executorService = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("SignificantTermsStream"));
   }
 
   public List<TupleStream> children() {
     return null;
   }
 
-  private List<String> getShardUrls() throws IOException {
-    try {
-      ZkStateReader zkStateReader = cloudSolrClient.getZkStateReader();
-
-      Collection<Slice> slices = CloudSolrStream.getSlices(this.collection, zkStateReader, false);
-
-      ClusterState clusterState = zkStateReader.getClusterState();
-      Set<String> liveNodes = clusterState.getLiveNodes();
-
-      List<String> baseUrls = new ArrayList<>();
-      for(Slice slice : slices) {
-        Collection<Replica> replicas = slice.getReplicas();
-        List<Replica> shuffler = new ArrayList<>();
-        for(Replica replica : replicas) {
-          if(replica.getState() == Replica.State.ACTIVE && liveNodes.contains(replica.getNodeName())) {
-            shuffler.add(replica);
-          }
-        }
-
-        Collections.shuffle(shuffler, new Random());
-        Replica rep = shuffler.get(0);
-        ZkCoreNodeProps zkProps = new ZkCoreNodeProps(rep);
-        String url = zkProps.getCoreUrl();
-        baseUrls.add(url);
-      }
-
-      return baseUrls;
-
-    } catch (Exception e) {
-      throw new IOException(e);
-    }
-  }
-
   private List<Future<NamedList>> callShards(List<String> baseUrls) throws IOException {
 
     List<Future<NamedList>> futures = new ArrayList<>();
@@ -326,7 +289,7 @@ public class SignificantTermsStream extends TupleStream implements Expressible{
         Map<String, int[]> mergeFreqs = new HashMap<>();
         long numDocs = 0;
         long resultCount = 0;
-        for (Future<NamedList> getTopTermsCall : callShards(getShardUrls())) {
+        for (Future<NamedList> getTopTermsCall : callShards(getShards(zkHost, collection, streamContext))) {
           NamedList resp = getTopTermsCall.get();
 
           List<String> terms = (List<String>)resp.get("sterms");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/682c6a7d/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TupleStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TupleStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TupleStream.java
index 49a806f..ceea6af 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TupleStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TupleStream.java
@@ -19,9 +19,16 @@ package org.apache.solr.client.solrj.io.stream;
 import java.io.Closeable;
 import java.io.IOException;
 import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
+import java.util.Random;
+import java.util.Set;
 import java.util.UUID;
+import java.util.Map;
 
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.io.Tuple;
 import org.apache.solr.client.solrj.io.comp.StreamComparator;
 import org.apache.solr.client.solrj.io.stream.expr.Explanation;
@@ -29,6 +36,14 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
 import org.apache.solr.common.IteratorWriter;
 import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.Aliases;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.util.StrUtils;
 
 
 public abstract class TupleStream implements Closeable, Serializable, MapWriter {
@@ -84,4 +99,83 @@ public abstract class TupleStream implements Closeable, Serializable, MapWriter
   public UUID getStreamNodeId(){
     return streamNodeId;
   }
+
+  public static List<String> getShards(String zkHost,
+                                       String collection,
+                                       StreamContext streamContext)
+      throws IOException {
+    Map<String, List<String>> shardsMap = null;
+    List<String> shards = new ArrayList();
+
+    if(streamContext != null) {
+      shardsMap = (Map<String, List<String>>)streamContext.get("shards");
+    }
+
+    if(shardsMap != null) {
+      //Manual Sharding
+      shards = shardsMap.get(collection);
+    } else {
+      //SolrCloud Sharding
+      CloudSolrClient cloudSolrClient = streamContext.getSolrClientCache().getCloudSolrClient(zkHost);
+      ZkStateReader zkStateReader = cloudSolrClient.getZkStateReader();
+      ClusterState clusterState = zkStateReader.getClusterState();
+      Collection<Slice> slices = getSlices(collection, zkStateReader, true);
+      Set<String> liveNodes = clusterState.getLiveNodes();
+      for(Slice slice : slices) {
+        Collection<Replica> replicas = slice.getReplicas();
+        List<Replica> shuffler = new ArrayList<>();
+        for(Replica replica : replicas) {
+          if(replica.getState() == Replica.State.ACTIVE && liveNodes.contains(replica.getNodeName()))
+            shuffler.add(replica);
+        }
+
+        Collections.shuffle(shuffler, new Random());
+        Replica rep = shuffler.get(0);
+        ZkCoreNodeProps zkProps = new ZkCoreNodeProps(rep);
+        String url = zkProps.getCoreUrl();
+        shards.add(url);
+      }
+    }
+
+    return shards;
+  }
+
+  public static Collection<Slice> getSlices(String collectionName,
+                                            ZkStateReader zkStateReader,
+                                            boolean checkAlias) throws IOException {
+    ClusterState clusterState = zkStateReader.getClusterState();
+
+    Map<String, DocCollection> collectionsMap = clusterState.getCollectionsMap();
+
+    // Check collection case sensitive
+    if(collectionsMap.containsKey(collectionName)) {
+      return collectionsMap.get(collectionName).getActiveSlices();
+    }
+
+    // Check collection case insensitive
+    for(String collectionMapKey : collectionsMap.keySet()) {
+      if(collectionMapKey.equalsIgnoreCase(collectionName)) {
+        return collectionsMap.get(collectionMapKey).getActiveSlices();
+      }
+    }
+
+    if(checkAlias) {
+      // check for collection alias
+      Aliases aliases = zkStateReader.getAliases();
+      String alias = aliases.getCollectionAlias(collectionName);
+      if (alias != null) {
+        Collection<Slice> slices = new ArrayList<>();
+
+        List<String> aliasList = StrUtils.splitSmart(alias, ",", true);
+        for (String aliasCollectionName : aliasList) {
+          // Add all active slices for this alias collection
+          slices.addAll(collectionsMap.get(aliasCollectionName).getActiveSlices());
+        }
+
+        return slices;
+      }
+    }
+
+    throw new IOException("Slices not found for " + collectionName);
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/682c6a7d/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 30b7056..c61e443 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
@@ -335,7 +335,7 @@ public class StreamExpressionTest extends SolrCloudTestCase {
     tuples = getTuples(stream);
     
     assert(tuples.size() == 4);
-    assertOrder(tuples, 4,3,1,2);
+    assertOrder(tuples, 4, 3, 1, 2);
     
     // Basic w/multi comp
     expression = StreamExpressionParser.parse("unique(search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\"), over=\"a_f, a_i\")");
@@ -1577,7 +1577,7 @@ public class StreamExpressionTest extends SolrCloudTestCase {
 
     List<Tuple> tuples = getTuples(pstream);
     assert(tuples.size() == 5);
-    assertOrder(tuples, 0,1,3,4,6);
+    assertOrder(tuples, 0, 1, 3, 4, 6);
 
     //Test the eofTuples
 
@@ -4712,8 +4712,6 @@ public class StreamExpressionTest extends SolrCloudTestCase {
   @Test
   public void testSignificantTermsStream() throws Exception {
 
-    Assume.assumeTrue(!useAlias);
-
     UpdateRequest updateRequest = new UpdateRequest();
     for (int i = 0; i < 5000; i++) {
       updateRequest.add(id, "a"+i, "test_t", "a b c d m l");
@@ -4742,106 +4740,186 @@ public class StreamExpressionTest extends SolrCloudTestCase {
 
     StreamFactory factory = new StreamFactory()
         .withCollectionZkHost("collection1", cluster.getZkServer().getZkAddress())
+        .withDefaultZkHost(cluster.getZkServer().getZkAddress())
         .withFunctionName("significantTerms", SignificantTermsStream.class);
 
-    String significantTerms = "significantTerms(collection1, q=\"id:a*\",  field=\"test_t\", limit=3, minTermLength=1, maxDocFreq=\".5\")";
-    stream = factory.constructStream(significantTerms);
-    tuples = getTuples(stream);
+    StreamContext streamContext = new StreamContext();
+    SolrClientCache cache = new SolrClientCache();
+    streamContext.setSolrClientCache(cache);
+    try {
 
-    assert(tuples.size() == 3);
-    assertTrue(tuples.get(0).get("term").equals("l"));
-    assertTrue(tuples.get(0).getLong("background") == 5000);
-    assertTrue(tuples.get(0).getLong("foreground") == 5000);
+      String significantTerms = "significantTerms(collection1, q=\"id:a*\",  field=\"test_t\", limit=3, minTermLength=1, maxDocFreq=\".5\")";
+      stream = factory.constructStream(significantTerms);
+      stream.setStreamContext(streamContext);
+      tuples = getTuples(stream);
 
+      assert (tuples.size() == 3);
+      assertTrue(tuples.get(0).get("term").equals("l"));
+      assertTrue(tuples.get(0).getLong("background") == 5000);
+      assertTrue(tuples.get(0).getLong("foreground") == 5000);
 
-    assertTrue(tuples.get(1).get("term").equals("m"));
-    assertTrue(tuples.get(1).getLong("background") == 5500);
-    assertTrue(tuples.get(1).getLong("foreground") == 5000);
 
-    assertTrue(tuples.get(2).get("term").equals("d"));
-    assertTrue(tuples.get(2).getLong("background") == 5600);
-    assertTrue(tuples.get(2).getLong("foreground") == 5000);
+      assertTrue(tuples.get(1).get("term").equals("m"));
+      assertTrue(tuples.get(1).getLong("background") == 5500);
+      assertTrue(tuples.get(1).getLong("foreground") == 5000);
 
-    //Test maxDocFreq
-    significantTerms = "significantTerms(collection1, q=\"id:a*\",  field=\"test_t\", limit=3, maxDocFreq=2650, minTermLength=1)";
-    stream = factory.constructStream(significantTerms);
-    tuples = getTuples(stream);
+      assertTrue(tuples.get(2).get("term").equals("d"));
+      assertTrue(tuples.get(2).getLong("background") == 5600);
+      assertTrue(tuples.get(2).getLong("foreground") == 5000);
 
-    assert(tuples.size() == 1);
-    assertTrue(tuples.get(0).get("term").equals("l"));
-    assertTrue(tuples.get(0).getLong("background") == 5000);
-    assertTrue(tuples.get(0).getLong("foreground") == 5000);
+      //Test maxDocFreq
+      significantTerms = "significantTerms(collection1, q=\"id:a*\",  field=\"test_t\", limit=3, maxDocFreq=2650, minTermLength=1)";
+      stream = factory.constructStream(significantTerms);
+      stream.setStreamContext(streamContext);
+      tuples = getTuples(stream);
 
-    //Test maxDocFreq percentage
+      assert (tuples.size() == 1);
+      assertTrue(tuples.get(0).get("term").equals("l"));
+      assertTrue(tuples.get(0).getLong("background") == 5000);
+      assertTrue(tuples.get(0).getLong("foreground") == 5000);
 
-    significantTerms = "significantTerms(collection1, q=\"id:a*\",  field=\"test_t\", limit=3, maxDocFreq=\".45\", minTermLength=1)";
-    stream = factory.constructStream(significantTerms);
-    tuples = getTuples(stream);
-    assert(tuples.size() == 1);
-    assertTrue(tuples.get(0).get("term").equals("l"));
-    assertTrue(tuples.get(0).getLong("background") == 5000);
-    assertTrue(tuples.get(0).getLong("foreground") == 5000);
+      //Test maxDocFreq percentage
 
+      significantTerms = "significantTerms(collection1, q=\"id:a*\",  field=\"test_t\", limit=3, maxDocFreq=\".45\", minTermLength=1)";
+      stream = factory.constructStream(significantTerms);
+      stream.setStreamContext(streamContext);
+      tuples = getTuples(stream);
+      assert (tuples.size() == 1);
+      assertTrue(tuples.get(0).get("term").equals("l"));
+      assertTrue(tuples.get(0).getLong("background") == 5000);
+      assertTrue(tuples.get(0).getLong("foreground") == 5000);
 
-    //Test min doc freq
-    significantTerms = "significantTerms(collection1, q=\"id:a*\",  field=\"test_t\", limit=3, minDocFreq=\"2700\", minTermLength=1, maxDocFreq=\".5\")";
-    stream = factory.constructStream(significantTerms);
-    tuples = getTuples(stream);
 
-    assert(tuples.size() == 3);
+      //Test min doc freq
+      significantTerms = "significantTerms(collection1, q=\"id:a*\",  field=\"test_t\", limit=3, minDocFreq=\"2700\", minTermLength=1, maxDocFreq=\".5\")";
+      stream = factory.constructStream(significantTerms);
+      stream.setStreamContext(streamContext);
+      tuples = getTuples(stream);
 
-    assertTrue(tuples.get(0).get("term").equals("m"));
-    assertTrue(tuples.get(0).getLong("background") == 5500);
-    assertTrue(tuples.get(0).getLong("foreground") == 5000);
+      assert (tuples.size() == 3);
 
-    assertTrue(tuples.get(1).get("term").equals("d"));
-    assertTrue(tuples.get(1).getLong("background") == 5600);
-    assertTrue(tuples.get(1).getLong("foreground") == 5000);
+      assertTrue(tuples.get(0).get("term").equals("m"));
+      assertTrue(tuples.get(0).getLong("background") == 5500);
+      assertTrue(tuples.get(0).getLong("foreground") == 5000);
 
-    assertTrue(tuples.get(2).get("term").equals("c"));
-    assertTrue(tuples.get(2).getLong("background") == 5900);
-    assertTrue(tuples.get(2).getLong("foreground") == 5000);
+      assertTrue(tuples.get(1).get("term").equals("d"));
+      assertTrue(tuples.get(1).getLong("background") == 5600);
+      assertTrue(tuples.get(1).getLong("foreground") == 5000);
 
+      assertTrue(tuples.get(2).get("term").equals("c"));
+      assertTrue(tuples.get(2).getLong("background") == 5900);
+      assertTrue(tuples.get(2).getLong("foreground") == 5000);
 
-    //Test min doc freq percent
-    significantTerms = "significantTerms(collection1, q=\"id:a*\",  field=\"test_t\", limit=3, minDocFreq=\".478\", minTermLength=1, maxDocFreq=\".5\")";
-    stream = factory.constructStream(significantTerms);
-    tuples = getTuples(stream);
 
-    assert(tuples.size() == 1);
+      //Test min doc freq percent
+      significantTerms = "significantTerms(collection1, q=\"id:a*\",  field=\"test_t\", limit=3, minDocFreq=\".478\", minTermLength=1, maxDocFreq=\".5\")";
+      stream = factory.constructStream(significantTerms);
+      stream.setStreamContext(streamContext);
+      tuples = getTuples(stream);
 
-    assertTrue(tuples.get(0).get("term").equals("c"));
-    assertTrue(tuples.get(0).getLong("background") == 5900);
-    assertTrue(tuples.get(0).getLong("foreground") == 5000);
+      assert (tuples.size() == 1);
 
+      assertTrue(tuples.get(0).get("term").equals("c"));
+      assertTrue(tuples.get(0).getLong("background") == 5900);
+      assertTrue(tuples.get(0).getLong("foreground") == 5000);
 
-    //Test limit
+      //Test limit
 
-    significantTerms = "significantTerms(collection1, q=\"id:a*\",  field=\"test_t\", limit=2, minDocFreq=\"2700\", minTermLength=1, maxDocFreq=\".5\")";
-    stream = factory.constructStream(significantTerms);
-    tuples = getTuples(stream);
+      significantTerms = "significantTerms(collection1, q=\"id:a*\",  field=\"test_t\", limit=2, minDocFreq=\"2700\", minTermLength=1, maxDocFreq=\".5\")";
+      stream = factory.constructStream(significantTerms);
+      stream.setStreamContext(streamContext);
+      tuples = getTuples(stream);
 
-    assert(tuples.size() == 2);
+      assert (tuples.size() == 2);
 
-    assertTrue(tuples.get(0).get("term").equals("m"));
-    assertTrue(tuples.get(0).getLong("background") == 5500);
-    assertTrue(tuples.get(0).getLong("foreground") == 5000);
+      assertTrue(tuples.get(0).get("term").equals("m"));
+      assertTrue(tuples.get(0).getLong("background") == 5500);
+      assertTrue(tuples.get(0).getLong("foreground") == 5000);
 
-    assertTrue(tuples.get(1).get("term").equals("d"));
-    assertTrue(tuples.get(1).getLong("background") == 5600);
-    assertTrue(tuples.get(1).getLong("foreground") == 5000);
+      assertTrue(tuples.get(1).get("term").equals("d"));
+      assertTrue(tuples.get(1).getLong("background") == 5600);
+      assertTrue(tuples.get(1).getLong("foreground") == 5000);
 
-    //Test term length
+      //Test term length
 
-    significantTerms = "significantTerms(collection1, q=\"id:a*\",  field=\"test_t\", limit=2, minDocFreq=\"2700\", minTermLength=2)";
-    stream = factory.constructStream(significantTerms);
-    tuples = getTuples(stream);
-    assert(tuples.size() == 0);
+      significantTerms = "significantTerms(collection1, q=\"id:a*\",  field=\"test_t\", limit=2, minDocFreq=\"2700\", minTermLength=2)";
+      stream = factory.constructStream(significantTerms);
+      stream.setStreamContext(streamContext);
+      tuples = getTuples(stream);
+      assert (tuples.size() == 0);
 
-  }
+
+      //Test with shards parameter
+      List<String> shardUrls = TupleStream.getShards(cluster.getZkServer().getZkAddress(), COLLECTIONORALIAS, streamContext);
+
+      Map<String, List<String>> shardsMap = new HashMap();
+      shardsMap.put("myCollection", shardUrls);
+      StreamContext context = new StreamContext();
+      context.put("shards", shardsMap);
+      context.setSolrClientCache(cache);
+      significantTerms = "significantTerms(myCollection, q=\"id:a*\",  field=\"test_t\", limit=2, minDocFreq=\"2700\", minTermLength=1, maxDocFreq=\".5\")";
+      stream = factory.constructStream(significantTerms);
+      stream.setStreamContext(context);
+      tuples = getTuples(stream);
+
+      assert (tuples.size() == 2);
+
+      assertTrue(tuples.get(0).get("term").equals("m"));
+      assertTrue(tuples.get(0).getLong("background") == 5500);
+      assertTrue(tuples.get(0).getLong("foreground") == 5000);
+
+      assertTrue(tuples.get(1).get("term").equals("d"));
+      assertTrue(tuples.get(1).getLong("background") == 5600);
+      assertTrue(tuples.get(1).getLong("foreground") == 5000);
+
+      //Execersise the /stream hander
+
+      //Add the shards http parameter for the myCollection
+      StringBuilder buf = new StringBuilder();
+      for (String shardUrl : shardUrls) {
+        if (buf.length() > 0) {
+          buf.append(",");
+        }
+        buf.append(shardUrl);
+      }
+
+      ModifiableSolrParams solrParams = new ModifiableSolrParams();
+      solrParams.add("qt", "/stream");
+      solrParams.add("expr", significantTerms);
+      solrParams.add("myCollection.shards", buf.toString());
+      SolrStream solrStream = new SolrStream(shardUrls.get(0), solrParams);
+      tuples = getTuples(solrStream);
+      assert (tuples.size() == 2);
+
+      assertTrue(tuples.get(0).get("term").equals("m"));
+      assertTrue(tuples.get(0).getLong("background") == 5500);
+      assertTrue(tuples.get(0).getLong("foreground") == 5000);
+
+      assertTrue(tuples.get(1).get("term").equals("d"));
+      assertTrue(tuples.get(1).getLong("background") == 5600);
+      assertTrue(tuples.get(1).getLong("foreground") == 5000);
+
+      //Add a negative test to prove that it cannot find slices if shards parameter is removed
+
+      try {
+        ModifiableSolrParams solrParamsBad = new ModifiableSolrParams();
+        solrParamsBad.add("qt", "/stream");
+        solrParamsBad.add("expr", significantTerms);
+        solrStream = new SolrStream(shardUrls.get(0), solrParamsBad);
+        tuples = getTuples(solrStream);
+        throw new Exception("Exception should have been thrown above");
+      } catch (IOException e) {
+        assertTrue(e.getMessage().contains("Slices not found for myCollection"));
+      }
+    } finally {
+      cache.close();
+    }
 
 
 
+
+  }
+
   @Test
   public void testComplementStream() throws Exception {
 
@@ -4920,12 +4998,16 @@ public class StreamExpressionTest extends SolrCloudTestCase {
   }
 
   protected List<Tuple> getTuples(TupleStream tupleStream) throws IOException {
-    tupleStream.open();
     List<Tuple> tuples = new ArrayList<Tuple>();
-    for(Tuple t = tupleStream.read(); !t.EOF; t = tupleStream.read()) {
-      tuples.add(t);
+
+    try {
+      tupleStream.open();
+      for (Tuple t = tupleStream.read(); !t.EOF; t = tupleStream.read()) {
+        tuples.add(t);
+      }
+    } finally {
+      tupleStream.close();
     }
-    tupleStream.close();
     return tuples;
   }
   protected boolean assertOrder(List<Tuple> tuples, int... ids) throws Exception {


[43/43] lucene-solr:feature/autoscaling: SOLR-8045: Fix smokeTestRelease.py from precommit

Posted by sh...@apache.org.
SOLR-8045: Fix smokeTestRelease.py from precommit


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

Branch: refs/heads/feature/autoscaling
Commit: faeb1fe8c16f9e02aa5c3bba295bc24325b94a07
Parents: abec54b
Author: Cao Manh Dat <da...@apache.org>
Authored: Tue Mar 14 08:30:38 2017 +0700
Committer: Cao Manh Dat <da...@apache.org>
Committed: Tue Mar 14 08:30:38 2017 +0700

----------------------------------------------------------------------
 dev-tools/scripts/smokeTestRelease.py | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/faeb1fe8/dev-tools/scripts/smokeTestRelease.py
----------------------------------------------------------------------
diff --git a/dev-tools/scripts/smokeTestRelease.py b/dev-tools/scripts/smokeTestRelease.py
index b0e76e9..9de490a 100644
--- a/dev-tools/scripts/smokeTestRelease.py
+++ b/dev-tools/scripts/smokeTestRelease.py
@@ -856,7 +856,7 @@ def testSolrExample(unpackPath, javaPath, isSrc):
       print('FAILED: response is:\n%s' % s)
       raise RuntimeError('query on solr example instance failed')
     s = load('http://localhost:8983/v2/cores')
-    if s.find('"responseHeader":{"status":0,"QTime":1}') == -1:
+    if s.find('"responseHeader":{"status":0') == -1:
       print('FAILED: response is:\n%s' % s)
       raise RuntimeError('query api v2 on solr example instance failed')
   finally:


[37/43] lucene-solr:feature/autoscaling: SOLR-10079: Increasing threadpool termination to 15s, clearIndex() to do clean replica indexes as well

Posted by sh...@apache.org.
SOLR-10079: Increasing threadpool termination to 15s, clearIndex() to do clean replica indexes as well


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

Branch: refs/heads/feature/autoscaling
Commit: d5181ec8e5bbc8932f606caaf281ff6913a1537d
Parents: b64382b
Author: Ishan Chattopadhyaya <is...@apache.org>
Authored: Mon Mar 13 16:55:50 2017 +0530
Committer: Ishan Chattopadhyaya <is...@apache.org>
Committed: Mon Mar 13 16:56:29 2017 +0530

----------------------------------------------------------------------
 .../solr/update/TestInPlaceUpdatesDistrib.java  | 89 +++++++++++++++-----
 1 file changed, 68 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5181ec8/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java
index 4c90bc6..bb0ab9a 100644
--- a/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java
+++ b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java
@@ -151,15 +151,18 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
         "docValues",Boolean.TRUE));
 
     // Do the tests now:
-    reorderedDBQIndividualReplicaTest();
-    testDBQUsingUpdatedFieldFromDroppedUpdate();
-    outOfOrderDBQsTest();
     docValuesUpdateTest();
     ensureRtgWorksWithPartialUpdatesTest();
-    delayedReorderingFetchesMissingUpdateFromLeaderTest();
     outOfOrderUpdatesIndividualReplicaTest();
-    outOfOrderDeleteUpdatesIndividualReplicaTest();
-    reorderedDBQsWithInPlaceUpdatesShouldNotThrowReplicaInLIRTest();
+    delayedReorderingFetchesMissingUpdateFromLeaderTest();
+    updatingDVsInAVeryOldSegment();
+
+    // TODO Should we combine all/some of these into a single test, so as to cut down on execution time?
+    reorderedDBQIndividualReplicaTest();
+    reorderedDeletesTest();
+    reorderedDBQsSimpleTest();
+    reorderedDBQsResurrectionTest();
+    reorderedDBQsUsingUpdatedValueFromADroppedUpdate();
   }
   
   private void mapReplicasToClients() throws KeeperException, InterruptedException {
@@ -195,7 +198,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
   final int NUM_RETRIES = 100, WAIT_TIME = 10;
 
   // The following should work: full update to doc 0, in-place update for doc 0, delete doc 0
-  private void outOfOrderDBQsTest() throws Exception {
+  private void reorderedDBQsSimpleTest() throws Exception {
     
     clearIndex();
     commit();
@@ -243,7 +246,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
     }
     
     threadpool.shutdown();
-    assertTrue("Thread pool didn't terminate within 10 secs", threadpool.awaitTermination(10, TimeUnit.SECONDS));
+    assertTrue("Thread pool didn't terminate within 15 secs", threadpool.awaitTermination(15, TimeUnit.SECONDS));
     
     // assert all requests were successful
     for (Future<UpdateResponse> resp: updateResponses) {
@@ -256,7 +259,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
       assertNull("This doc was supposed to have been deleted, but was: " + doc, doc);
     }
 
-    log.info("outOfOrderDeleteUpdatesIndividualReplicaTest: This test passed fine...");
+    log.info("reorderedDBQsSimpleTest: This test passed fine...");
     clearIndex();
     commit();
   }
@@ -294,7 +297,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
     }
 
     threadpool.shutdown();
-    assertTrue("Thread pool didn't terminate within 10 secs", threadpool.awaitTermination(10, TimeUnit.SECONDS));
+    assertTrue("Thread pool didn't terminate within 15 secs", threadpool.awaitTermination(15, TimeUnit.SECONDS));
 
     // assert all requests were successful
     for (Future<UpdateResponse> resp: updateResponses) {
@@ -391,6 +394,36 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
   }
 
   /**
+   * Ingest many documents, keep committing. Then update a document from a very old segment.
+   */
+  private void updatingDVsInAVeryOldSegment() throws Exception {
+    clearIndex();
+    commit();
+
+    String id = String.valueOf(Integer.MAX_VALUE);
+    index("id", id, "inplace_updatable_float", "1", "title_s", "newtitle");
+
+    // create 10 more segments
+    for (int i=0; i<10; i++) {
+      buildRandomIndex(101.0F, Collections.emptyList());
+    }
+
+    index("id", id, "inplace_updatable_float", map("inc", "1"));
+
+    for (SolrClient client: new SolrClient[] {LEADER, NONLEADERS.get(0), NONLEADERS.get(1)}) {
+      assertEquals("newtitle", client.getById(id).get("title_s"));
+      assertEquals(2.0f, client.getById(id).get("inplace_updatable_float"));
+    }
+    commit();
+    for (SolrClient client: new SolrClient[] {LEADER, NONLEADERS.get(0), NONLEADERS.get(1)}) {
+      assertEquals("newtitle", client.getById(id).get("title_s"));
+      assertEquals(2.0f, client.getById(id).get("inplace_updatable_float"));
+    }
+
+    log.info("updatingDVsInAVeryOldSegment: This test passed fine...");
+  }
+
+  /**
    * Retries the specified 'req' against each SolrClient in "clients" untill the expected number of 
    * results are returned, at which point the results are verified using assertDocIdsAndValuesInResults
    *
@@ -610,7 +643,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
     }
     
     threadpool.shutdown();
-    assertTrue("Thread pool didn't terminate within 10 secs", threadpool.awaitTermination(10, TimeUnit.SECONDS));
+    assertTrue("Thread pool didn't terminate within 15 secs", threadpool.awaitTermination(15, TimeUnit.SECONDS));
 
     // assert all requests were successful
     for (Future<UpdateResponse> resp: updateResponses) {
@@ -633,7 +666,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
   }
   
   // The following should work: full update to doc 0, in-place update for doc 0, delete doc 0
-  private void outOfOrderDeleteUpdatesIndividualReplicaTest() throws Exception {
+  private void reorderedDeletesTest() throws Exception {
     
     clearIndex();
     commit();
@@ -680,7 +713,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
     }
     
     threadpool.shutdown();
-    assertTrue("Thread pool didn't terminate within 10 secs", threadpool.awaitTermination(10, TimeUnit.SECONDS));
+    assertTrue("Thread pool didn't terminate within 15 secs", threadpool.awaitTermination(15, TimeUnit.SECONDS));
 
     // assert all requests were successful
     for (Future<UpdateResponse> resp: updateResponses) {
@@ -693,7 +726,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
       assertNull("This doc was supposed to have been deleted, but was: " + doc, doc);
     }
 
-    log.info("outOfOrderDeleteUpdatesIndividualReplicaTest: This test passed fine...");
+    log.info("reorderedDeletesTest: This test passed fine...");
     clearIndex();
     commit();
   }
@@ -707,7 +740,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
         DBQ(q=val:10, v=4)
         DV(id=x, val=5, ver=3)
    */
-  private void reorderedDBQsWithInPlaceUpdatesShouldNotThrowReplicaInLIRTest() throws Exception {
+  private void reorderedDBQsResurrectionTest() throws Exception {
     clearIndex();
     commit();
 
@@ -754,7 +787,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
     }
     
     threadpool.shutdown();
-    assertTrue("Thread pool didn't terminate within 10 secs", threadpool.awaitTermination(10, TimeUnit.SECONDS));
+    assertTrue("Thread pool didn't terminate within 15 secs", threadpool.awaitTermination(15, TimeUnit.SECONDS));
 
     int successful = 0;
     for (Future<UpdateResponse> resp: updateResponses) {
@@ -794,7 +827,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
       assertEquals("Client: "+((HttpSolrClient)client).getBaseURL(), 5, doc.getFieldValue(field));
     }
 
-    log.info("reorderedDBQsWithInPlaceUpdatesShouldNotThrowReplicaInLIRTest: This test passed fine...");
+    log.info("reorderedDBQsResurrectionTest: This test passed fine...");
     clearIndex();
     commit();
   }
@@ -829,7 +862,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
     }
 
     threadpool.shutdown();
-    assertTrue("Thread pool didn't terminate within 10 secs", threadpool.awaitTermination(15, TimeUnit.SECONDS));
+    assertTrue("Thread pool didn't terminate within 15 secs", threadpool.awaitTermination(15, TimeUnit.SECONDS));
 
     commit();
 
@@ -1104,7 +1137,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
    * inp(id=1,inpfield=14,prevVersion=2,version=3) // will wait till timeout, and then fetch a "not found" from leader
    * dbq("inp:14",version=4)
    */
-  private void testDBQUsingUpdatedFieldFromDroppedUpdate() throws Exception {
+  private void reorderedDBQsUsingUpdatedValueFromADroppedUpdate() throws Exception {
     clearIndex();
     commit();
     
@@ -1161,7 +1194,21 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
       assertNull(client.getById("1", params("distrib", "false")));
     }
 
-    log.info("testDBQUsingUpdatedFieldFromDroppedUpdate: This test passed fine...");
+    log.info("reorderedDBQsUsingUpdatedValueFromADroppedUpdate: This test passed fine...");
+  }
+
+  @Override
+  public void clearIndex() {
+    super.clearIndex();
+    try {
+      for (SolrClient client: new SolrClient[] {LEADER, NONLEADERS.get(0), NONLEADERS.get(1)}) {
+        if (client != null) {
+          client.request(simulatedDeleteRequest("*:*", -Long.MAX_VALUE));
+          client.commit();
+        }
+      }
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
   }
-  
 }


[41/43] lucene-solr:feature/autoscaling: SOLR-10269 MetricHandler JSON output was incorrect.

Posted by sh...@apache.org.
SOLR-10269 MetricHandler JSON output was incorrect.


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

Branch: refs/heads/feature/autoscaling
Commit: e3a0b428fd7dd8747a6b48ef165300ebb23b3198
Parents: c8bad8c
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Mon Mar 13 20:29:22 2017 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Mon Mar 13 20:30:27 2017 +0100

----------------------------------------------------------------------
 solr/CHANGES.txt                                         |  2 ++
 .../src/java/org/apache/solr/util/stats/MetricUtils.java |  7 ++++---
 .../apache/solr/handler/admin/MetricsHandlerTest.java    | 11 +++++++----
 3 files changed, 13 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e3a0b428/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 1469d3e..63424dd 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -235,6 +235,8 @@ Bug Fixes
 
 * SOLR-9838: "inc" atomic update doesn't respect default field value (hoss, Amrit Sarkar, Ishan Chattopadhyaya)
 
+* SOLR-10269: MetricsHandler JSON output incorrect. (ab)
+
 Optimizations
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e3a0b428/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java b/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java
index 5a7c680..70fd467 100644
--- a/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java
+++ b/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java
@@ -38,6 +38,7 @@ import com.codahale.metrics.Snapshot;
 import com.codahale.metrics.Timer;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.metrics.AggregateMetric;
 
 /**
@@ -117,12 +118,12 @@ public class MetricUtils {
                                       MetricFilter mustMatchFilter, boolean skipHistograms,
                                       boolean skipAggregateValues,
                                       Map<String, Object> metadata) {
-    NamedList result = new NamedList();
+    NamedList result = new SimpleOrderedMap();
     toNamedMaps(registry, shouldMatchFilters, mustMatchFilter, skipHistograms, skipAggregateValues, (k, v) -> {
-      result.add(k, new NamedList(v));
+      result.add(k, v);
     });
     if (metadata != null && !metadata.isEmpty()) {
-      result.add("_metadata_", new NamedList(metadata));
+      result.add("_metadata_", metadata);
     }
     return result;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e3a0b428/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 a1b29db..2fd7e9c 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
@@ -17,6 +17,8 @@
 
 package org.apache.solr.handler.admin;
 
+import java.util.Map;
+
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.util.NamedList;
@@ -48,13 +50,14 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 {
     NamedList nl = (NamedList) values.get("solr.core.collection1");
     assertNotNull(nl);
     assertNotNull(nl.get("SEARCHER.new.errors")); // counter type
-    assertNotNull(((NamedList) nl.get("SEARCHER.new.errors")).get("count"));
-    assertEquals(0L, ((NamedList) nl.get("SEARCHER.new.errors")).get("count"));
+    assertNotNull(((Map) nl.get("SEARCHER.new.errors")).get("count"));
+    // response wasn't serialized so we get here whatever MetricUtils produced instead of NamedList
+    assertEquals(0L, ((Map) nl.get("SEARCHER.new.errors")).get("count"));
     nl = (NamedList) values.get("solr.node");
     assertNotNull(nl.get("CONTAINER.cores.loaded")); // int gauge
-    assertEquals(1, ((NamedList) nl.get("CONTAINER.cores.loaded")).get("value"));
+    assertEquals(1, ((Map) nl.get("CONTAINER.cores.loaded")).get("value"));
     assertNotNull(nl.get("ADMIN./admin/authorization.clientErrors")); // timer type
-    assertEquals(5, ((NamedList) nl.get("ADMIN./admin/authorization.clientErrors")).size());
+    assertEquals(5, ((Map) nl.get("ADMIN./admin/authorization.clientErrors")).size());
 
     resp = new SolrQueryResponse();
     handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", "group", "jvm,jetty"), resp);


[24/43] lucene-solr:feature/autoscaling: LUCENE-7700: Move throughput control and merge aborting out of IndexWriter's core.

Posted by sh...@apache.org.
LUCENE-7700: Move throughput control and merge aborting out of IndexWriter's core.


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

Branch: refs/heads/feature/autoscaling
Commit: 9540bc37583dfd4e995b893154039fcf031dc3c3
Parents: d2bf30d
Author: Dawid Weiss <dw...@apache.org>
Authored: Fri Mar 10 10:23:29 2017 +0100
Committer: Dawid Weiss <dw...@apache.org>
Committed: Fri Mar 10 10:23:29 2017 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   6 +
 .../lucene/index/ConcurrentMergeScheduler.java  |  75 ++++++--
 .../org/apache/lucene/index/IndexWriter.java    |  94 ++++------
 .../org/apache/lucene/index/MergePolicy.java    | 184 +++++++++++++++++--
 .../apache/lucene/index/MergeRateLimiter.java   | 177 +++++++-----------
 .../org/apache/lucene/index/MergeScheduler.java |  12 ++
 .../apache/lucene/index/NoMergeScheduler.java   |   7 +
 .../lucene/index/TestMergeRateLimiter.java      |   4 +-
 8 files changed, 358 insertions(+), 201 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9540bc37/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 4040945..b6ee4b8 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -117,6 +117,12 @@ API Changes
   instead of once all shard responses are present. (Simon Willnauer,
   Mike McCandless)
 
+* LUCENE-7700: A cleanup of merge throughput control logic. Refactored all the
+  code previously scattered throughout the IndexWriter and 
+  ConcurrentMergeScheduler into a more accessible set of public methods (see 
+  MergePolicy.OneMergeProgress, MergeScheduler.wrapForMerge and 
+  OneMerge.mergeInit). (Dawid Weiss, Mike McCandless).
+
 * LUCENE-7734: FieldType's copy constructor was widened to accept any IndexableFieldType.
   (David Smiley)
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9540bc37/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java b/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java
index 0dd0a4d..6e930c4 100644
--- a/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java
+++ b/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java
@@ -25,6 +25,11 @@ import java.util.Locale;
 import org.apache.lucene.index.MergePolicy.OneMerge;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FilterDirectory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.RateLimitedIndexOutput;
+import org.apache.lucene.store.RateLimiter;
 import org.apache.lucene.util.CollectionUtil;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.ThreadInterruptedException;
@@ -255,6 +260,36 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
     assert false: "merge thread " + currentThread + " was not found";
   }
 
+  @Override
+  public Directory wrapForMerge(OneMerge merge, Directory in) {
+    Thread mergeThread = Thread.currentThread();
+    if (!MergeThread.class.isInstance(mergeThread)) {
+      throw new AssertionError("wrapForMerge should be called from MergeThread. Current thread: "
+          + mergeThread);
+    }
+
+    // Return a wrapped Directory which has rate-limited output.
+    RateLimiter rateLimiter = ((MergeThread) mergeThread).rateLimiter;
+    return new FilterDirectory(in) {
+      @Override
+      public IndexOutput createOutput(String name, IOContext context) throws IOException {
+        ensureOpen();
+
+        // This Directory is only supposed to be used during merging,
+        // so all writes should have MERGE context, else there is a bug 
+        // somewhere that is failing to pass down the right IOContext:
+        assert context.context == IOContext.Context.MERGE: "got context=" + context.context;
+        
+        // Because rateLimiter is bound to a particular merge thread, this method should
+        // always be called from that context. Verify this.
+        assert mergeThread == Thread.currentThread() : "Not the same merge thread, current="
+          + Thread.currentThread() + ", expected=" + mergeThread;
+
+        return new RateLimitedIndexOutput(rateLimiter, in.createOutput(name, context));
+      }
+    };
+  }
+  
   /**
    * Called whenever the running merges have changed, to set merge IO limits.
    * This method sorts the merge threads by their merge size in
@@ -327,8 +362,9 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
         newMBPerSec = targetMBPerSec;
       }
 
-      double curMBPerSec = merge.rateLimiter.getMBPerSec();
-      
+      MergeRateLimiter rateLimiter = mergeThread.rateLimiter;
+      double curMBPerSec = rateLimiter.getMBPerSec();
+
       if (verbose()) {
         long mergeStartNS = merge.mergeStartNS;
         if (mergeStartNS == -1) {
@@ -339,11 +375,11 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
         message.append(String.format(Locale.ROOT, "merge thread %s estSize=%.1f MB (written=%.1f MB) runTime=%.1fs (stopped=%.1fs, paused=%.1fs) rate=%s\n",
                                      mergeThread.getName(),
                                      bytesToMB(merge.estimatedMergeBytes),
-                                     bytesToMB(merge.rateLimiter.totalBytesWritten),
+                                     bytesToMB(rateLimiter.getTotalBytesWritten()),
                                      nsToSec(now - mergeStartNS),
-                                     nsToSec(merge.rateLimiter.getTotalStoppedNS()),
-                                     nsToSec(merge.rateLimiter.getTotalPausedNS()),
-                                     rateToString(merge.rateLimiter.getMBPerSec())));
+                                     nsToSec(rateLimiter.getTotalStoppedNS()),
+                                     nsToSec(rateLimiter.getTotalPausedNS()),
+                                     rateToString(rateLimiter.getMBPerSec())));
 
         if (newMBPerSec != curMBPerSec) {
           if (newMBPerSec == 0.0) {
@@ -364,7 +400,7 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
         }
       }
 
-      merge.rateLimiter.setMBPerSec(newMBPerSec);
+      rateLimiter.setMBPerSec(newMBPerSec);
     }
     if (verbose()) {
       message(message.toString());
@@ -449,7 +485,7 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
     Thread currentThread = Thread.currentThread();
     int count = 0;
     for (MergeThread mergeThread : mergeThreads) {
-      if (currentThread != mergeThread && mergeThread.isAlive() && mergeThread.merge.rateLimiter.getAbort() == false) {
+      if (currentThread != mergeThread && mergeThread.isAlive() && mergeThread.merge.isAborted() == false) {
         count++;
       }
     }
@@ -497,8 +533,6 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
         return;
       }
 
-      updateIOThrottle(merge);
-
       boolean success = false;
       try {
         if (verbose()) {
@@ -507,14 +541,16 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
 
         // OK to spawn a new merge thread to handle this
         // merge:
-        final MergeThread merger = getMergeThread(writer, merge);
-        mergeThreads.add(merger);
+        final MergeThread newMergeThread = getMergeThread(writer, merge);
+        mergeThreads.add(newMergeThread);
+
+        updateIOThrottle(newMergeThread.merge, newMergeThread.rateLimiter);
 
         if (verbose()) {
-          message("    launch new thread [" + merger.getName() + "]");
+          message("    launch new thread [" + newMergeThread.getName() + "]");
         }
 
-        merger.start();
+        newMergeThread.start();
         updateMergeThreads();
 
         success = true;
@@ -598,16 +634,17 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
 
   /** Runs a merge thread to execute a single merge, then exits. */
   protected class MergeThread extends Thread implements Comparable<MergeThread> {
-
     final IndexWriter writer;
     final OneMerge merge;
+    final MergeRateLimiter rateLimiter;
 
     /** Sole constructor. */
     public MergeThread(IndexWriter writer, OneMerge merge) {
       this.writer = writer;
       this.merge = merge;
+      this.rateLimiter = new MergeRateLimiter(merge.getMergeProgress());
     }
-    
+
     @Override
     public int compareTo(MergeThread other) {
       // Larger merges sort first:
@@ -616,9 +653,7 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
 
     @Override
     public void run() {
-
       try {
-
         if (verbose()) {
           message("  merge thread: start");
         }
@@ -715,7 +750,7 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
   }
 
   /** Tunes IO throttle when a new merge starts. */
-  private synchronized void updateIOThrottle(OneMerge newMerge) throws IOException {
+  private synchronized void updateIOThrottle(OneMerge newMerge, MergeRateLimiter rateLimiter) throws IOException {
     if (doAutoIOThrottle == false) {
       return;
     }
@@ -794,7 +829,7 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
     } else {
       rate = targetMBPerSec;
     }
-    newMerge.rateLimiter.setMBPerSec(rate);
+    rateLimiter.setMBPerSec(rate);
     targetMBPerSecChanged();
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9540bc37/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
index da030ca..aa28d99 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -36,6 +36,7 @@ import java.util.Queue;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Collectors;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.codecs.Codec;
@@ -51,22 +52,18 @@ import org.apache.lucene.search.Sort;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
-import org.apache.lucene.store.FilterDirectory;
 import org.apache.lucene.store.FlushInfo;
 import org.apache.lucene.store.IOContext;
-import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.Lock;
 import org.apache.lucene.store.LockObtainFailedException;
 import org.apache.lucene.store.LockValidatingDirectoryWrapper;
 import org.apache.lucene.store.MMapDirectory;
 import org.apache.lucene.store.MergeInfo;
-import org.apache.lucene.store.RateLimitedIndexOutput;
 import org.apache.lucene.store.TrackingDirectoryWrapper;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.CloseableThreadLocal;
 import org.apache.lucene.util.Constants;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.InfoStream;
@@ -277,7 +274,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
 
   private final Directory directoryOrig;       // original user directory
   private final Directory directory;           // wrapped with additional checks
-  private final Directory mergeDirectory;      // wrapped with throttling: used for merging
   private final Analyzer analyzer;    // how to analyze text
 
   private final AtomicLong changeCount = new AtomicLong(); // increments every time a change is completed
@@ -353,8 +349,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
    *  card to make sure they can later charge you when you check out. */
   final AtomicLong pendingNumDocs = new AtomicLong();
 
-  final CloseableThreadLocal<MergeRateLimiter> rateLimiters = new CloseableThreadLocal<>();
-
   DirectoryReader getReader() throws IOException {
     return getReader(true, false);
   }
@@ -809,10 +803,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
       directoryOrig = d;
       directory = new LockValidatingDirectoryWrapper(d, writeLock);
 
-      // Directory we use for merging, so we can abort running merges, and so
-      // merge schedulers can optionally rate-limit per-merge IO:
-      mergeDirectory = addMergeRateLimiters(directory);
-
       analyzer = config.getAnalyzer();
       mergeScheduler = config.getMergeScheduler();
       mergeScheduler.setInfoStream(infoStream);
@@ -2212,8 +2202,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
     try {
       abortMerges();
 
-      rateLimiters.close();
-
       if (infoStream.isEnabled("IW")) {
         infoStream.message("IW", "rollback: done finish merges");
       }
@@ -2418,7 +2406,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
       if (infoStream.isEnabled("IW")) {
         infoStream.message("IW", "now abort pending merge " + segString(merge.segments));
       }
-      merge.rateLimiter.setAbort();
+      merge.setAborted();
       mergeFinish(merge);
     }
     pendingMerges.clear();
@@ -2427,7 +2415,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
       if (infoStream.isEnabled("IW")) {
         infoStream.message("IW", "now abort running merge " + segString(merge.segments));
       }
-      merge.rateLimiter.setAbort();
+      merge.setAborted();
     }
 
     // We wait here to make all merges stop.  It should not
@@ -2775,13 +2763,17 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
    * index.
    * 
    * <p>
-   * <b>NOTE:</b> this method merges all given {@link LeafReader}s in one
+   * <b>NOTE:</b> this merges all given {@link LeafReader}s in one
    * merge. If you intend to merge a large number of readers, it may be better
    * to call this method multiple times, each time with a small set of readers.
    * In principle, if you use a merge policy with a {@code mergeFactor} or
    * {@code maxMergeAtOnce} parameter, you should pass that many readers in one
    * call.
    * 
+   * <p>
+   * <b>NOTE:</b> this method does not call or make use of the {@link MergeScheduler},
+   * so any custom bandwidth throttling is at the moment ignored.
+   * 
    * @return The <a href="#sequence_number">sequence number</a>
    * for this operation
    *
@@ -2832,8 +2824,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
       SegmentMerger merger = new SegmentMerger(Arrays.asList(readers), info, infoStream, trackingDir,
                                                globalFieldNumberMap, 
                                                context);
-      
-      rateLimiters.set(new MergeRateLimiter(null));
 
       if (!merger.shouldMerge()) {
         return docWriter.deleteQueue.getNextSequenceNumber();
@@ -2864,7 +2854,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
       // Now create the compound file if needed
       if (useCompoundFile) {
         Collection<String> filesToDelete = infoPerCommit.files();
-        TrackingDirectoryWrapper trackingCFSDir = new TrackingDirectoryWrapper(mergeDirectory);
+        TrackingDirectoryWrapper trackingCFSDir = new TrackingDirectoryWrapper(directory);
         // TODO: unlike merge, on exception we arent sniping any trash cfs files here?
         // createCompoundFile tries to cleanup, but it might not always be able to...
         try {
@@ -3745,7 +3735,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
     // deleter.refresh() call that will remove any index
     // file that current segments does not reference), we
     // abort this merge
-    if (merge.rateLimiter.getAbort()) {
+    if (merge.isAborted()) {
       if (infoStream.isEnabled("IW")) {
         infoStream.message("IW", "commitMerge: skip: it was aborted");
       }
@@ -3905,8 +3895,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
 
     boolean success = false;
 
-    rateLimiters.set(merge.rateLimiter);
-
     final long t0 = System.currentTimeMillis();
 
     final MergePolicy mergePolicy = config.getMergePolicy();
@@ -3937,7 +3925,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
             if (infoStream.isEnabled("IW")) {
               infoStream.message("IW", "hit exception during merge");
             }
-          } else if (merge.rateLimiter.getAbort() == false && (merge.maxNumSegments != UNBOUNDED_MAX_MERGE_SEGMENTS || (!closed && !closing))) {
+          } else if (!merge.isAborted() && (merge.maxNumSegments != UNBOUNDED_MAX_MERGE_SEGMENTS || (!closed && !closing))) {
             // This merge (and, generally, any change to the
             // segments) may now enable new merges, so we call
             // merge policy & update pending merges.
@@ -3951,7 +3939,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
       tragicEvent(t, "merge");
     }
 
-    if (merge.info != null && merge.rateLimiter.getAbort() == false) {
+    if (merge.info != null && merge.isAborted() == false) {
       if (infoStream.isEnabled("IW")) {
         infoStream.message("IW", "merge time " + (System.currentTimeMillis()-t0) + " msec for " + merge.info.info.maxDoc() + " docs");
       }
@@ -3976,7 +3964,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
     assert merge.segments.size() > 0;
 
     if (stopMerges) {
-      merge.rateLimiter.setAbort();
+      merge.setAborted();
       throw new MergePolicy.MergeAbortedException("merge is aborted: " + segString(merge.segments));
     }
 
@@ -4087,7 +4075,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
       return;
     }
 
-    if (merge.rateLimiter.getAbort()) {
+    merge.mergeInit();
+
+    if (merge.isAborted()) {
       return;
     }
 
@@ -4239,9 +4229,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
    *  but without holding synchronized lock on IndexWriter
    *  instance */
   private int mergeMiddle(MergePolicy.OneMerge merge, MergePolicy mergePolicy) throws IOException {
+    merge.checkAborted();
 
-    merge.rateLimiter.checkAbort();
-
+    Directory mergeDirectory = config.getMergeScheduler().wrapForMerge(merge, directory);
     List<SegmentCommitInfo> sourceSegments = merge.segments;
     
     IOContext context = new IOContext(merge.getStoreMergeInfo());
@@ -4339,7 +4329,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
                                                      globalFieldNumberMap, 
                                                      context);
 
-      merge.rateLimiter.checkAbort();
+      merge.checkAborted();
 
       merge.mergeStartNS = System.nanoTime();
 
@@ -4354,11 +4344,20 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
 
       if (infoStream.isEnabled("IW")) {
         if (merger.shouldMerge()) {
+          String pauseInfo = merge.getMergeProgress().getPauseTimes().entrySet()
+            .stream()
+            .filter((e) -> e.getValue() > 0)
+            .map((e) -> String.format(Locale.ROOT, "%.1f sec %s", 
+                e.getValue() / 1000000000., 
+                e.getKey().name().toLowerCase(Locale.ROOT)))
+            .collect(Collectors.joining(", "));
+          if (!pauseInfo.isEmpty()) {
+            pauseInfo = " (" + pauseInfo + ")";
+          }
+
           long t1 = System.nanoTime();
           double sec = (t1-merge.mergeStartNS)/1000000000.;
           double segmentMB = (merge.info.sizeInBytes()/1024./1024.);
-          double stoppedSec = merge.rateLimiter.getTotalStoppedNS()/1000000000.;
-          double throttleSec = merge.rateLimiter.getTotalPausedNS()/1000000000.;
           infoStream.message("IW", "merge codec=" + codec + " maxDoc=" + merge.info.info.maxDoc() + "; merged segment has " +
                              (mergeState.mergeFieldInfos.hasVectors() ? "vectors" : "no vectors") + "; " +
                              (mergeState.mergeFieldInfos.hasNorms() ? "norms" : "no norms") + "; " + 
@@ -4367,10 +4366,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
                              (mergeState.mergeFieldInfos.hasProx() ? "freqs" : "no freqs") + "; " +
                              (mergeState.mergeFieldInfos.hasPointValues() ? "points" : "no points") + "; " +
                              String.format(Locale.ROOT,
-                                           "%.1f sec (%.1f sec stopped, %.1f sec paused) to merge segment [%.2f MB, %.2f MB/sec]",
+                                           "%.1f sec%s to merge segment [%.2f MB, %.2f MB/sec]",
                                            sec,
-                                           stoppedSec,
-                                           throttleSec,
+                                           pauseInfo,
                                            segmentMB,
                                            segmentMB / sec));
         } else {
@@ -4406,7 +4404,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
           success = true;
         } catch (Throwable t) {
           synchronized(this) {
-            if (merge.rateLimiter.getAbort()) {
+            if (merge.isAborted()) {
               // This can happen if rollback is called while we were building
               // our CFS -- fall through to logic below to remove the non-CFS
               // merged files:
@@ -4439,7 +4437,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
           // registered with IFD
           deleteNewFiles(filesToRemove);
 
-          if (merge.rateLimiter.getAbort()) {
+          if (merge.isAborted()) {
             if (infoStream.isEnabled("IW")) {
               infoStream.message("IW", "abort merge after building CFS");
             }
@@ -5063,30 +5061,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
     throw new IllegalArgumentException("number of documents in the index cannot exceed " + actualMaxDocs + " (current document count is " + pendingNumDocs.get() + "; added numDocs is " + addedNumDocs + ")");
   }
 
-  /** Wraps the incoming {@link Directory} so that we assign a per-thread
-   *  {@link MergeRateLimiter} to all created {@link IndexOutput}s. */
-  private Directory addMergeRateLimiters(Directory in) {
-    return new FilterDirectory(in) {
-      @Override
-      public IndexOutput createOutput(String name, IOContext context) throws IOException {
-        ensureOpen();
-
-        // Paranoia defense: if this trips we have a bug somewhere...
-        IndexWriter.this.ensureOpen(false);
-
-        // This Directory is only supposed to be used during merging,
-        // so all writes should have MERGE context, else there is a bug 
-        // somewhere that is failing to pass down the right IOContext:
-        assert context.context == IOContext.Context.MERGE: "got context=" + context.context;
-
-        MergeRateLimiter rateLimiter = rateLimiters.get();
-        assert rateLimiter != null;
-
-        return new RateLimitedIndexOutput(rateLimiter, in.createOutput(name, context));
-      }
-    };
-  }
-
   /** Returns the highest <a href="#sequence_number">sequence number</a> across
    *  all completed operations, or 0 if no operations have finished yet.  Still
    *  in-flight operations (in other threads) are not counted until they finish.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9540bc37/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java b/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
index dbf37df..d9a0ab8 100644
--- a/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
+++ b/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
@@ -19,12 +19,19 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.EnumMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.BooleanSupplier;
+import java.util.stream.Collectors;
 
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.MergeInfo;
-import org.apache.lucene.store.RateLimiter;
 
 /**
  * <p>Expert: a MergePolicy determines the sequence of
@@ -55,6 +62,125 @@ import org.apache.lucene.store.RateLimiter;
  * @lucene.experimental
  */
 public abstract class MergePolicy {
+  /**
+   * Progress and state for an executing merge. This class
+   * encapsulates the logic to pause and resume the merge thread
+   * or to abort the merge entirely.
+   * 
+   * @lucene.experimental */
+  public static class OneMergeProgress {
+    /** Reason for pausing the merge thread. */
+    public static enum PauseReason {
+      /** Stopped (because of throughput rate set to 0, typically). */
+      STOPPED,
+      /** Temporarily paused because of exceeded throughput rate. */
+      PAUSED,
+      /** Other reason. */
+      OTHER
+    };
+
+    private final ReentrantLock pauseLock = new ReentrantLock();
+    private final Condition pausing = pauseLock.newCondition();
+
+    /**
+     * Pause times (in nanoseconds) for each {@link PauseReason}.
+     */
+    private final EnumMap<PauseReason, AtomicLong> pauseTimesNS;
+    
+    private volatile boolean aborted;
+
+    /**
+     * This field is for sanity-check purposes only. Only the same thread that invoked
+     * {@link OneMerge#mergeInit()} is permitted to be calling 
+     * {@link #pauseNanos}. This is always verified at runtime. 
+     */
+    private Thread owner;
+
+    /** Creates a new merge progress info. */
+    public OneMergeProgress() {
+      // Place all the pause reasons in there immediately so that we can simply update values.
+      pauseTimesNS = new EnumMap<PauseReason,AtomicLong>(PauseReason.class);
+      for (PauseReason p : PauseReason.values()) {
+        pauseTimesNS.put(p, new AtomicLong());
+      }
+    }
+
+    /**
+     * Abort the merge this progress tracks at the next 
+     * possible moment.
+     */
+    public void abort() {
+      aborted = true;
+      wakeup(); // wakeup any paused merge thread.
+    }
+
+    /**
+     * Return the aborted state of this merge.
+     */
+    public boolean isAborted() {
+      return aborted;
+    }
+
+    /**
+     * Pauses the calling thread for at least <code>pauseNanos</code> nanoseconds
+     * unless the merge is aborted or the external condition returns <code>false</code>,
+     * in which case control returns immediately.
+     * 
+     * The external condition is required so that other threads can terminate the pausing immediately,
+     * before <code>pauseNanos</code> expires. We can't rely on just {@link Condition#awaitNanos(long)} alone
+     * because it can return due to spurious wakeups too.  
+     * 
+     * @param condition The pause condition that should return false if immediate return from this
+     *      method is needed. Other threads can wake up any sleeping thread by calling 
+     *      {@link #wakeup}, but it'd fall to sleep for the remainder of the requested time if this
+     *      condition 
+     */
+    public void pauseNanos(long pauseNanos, PauseReason reason, BooleanSupplier condition) throws InterruptedException {
+      if (Thread.currentThread() != owner) {
+        throw new RuntimeException("Only the merge owner thread can call pauseNanos(). This thread: "
+            + Thread.currentThread().getName() + ", owner thread: "
+            + owner);
+      }
+
+      long start = System.nanoTime();
+      AtomicLong timeUpdate = pauseTimesNS.get(reason);
+      pauseLock.lock();
+      try {
+        while (pauseNanos > 0 && !aborted && condition.getAsBoolean()) {
+          pauseNanos = pausing.awaitNanos(pauseNanos);
+        }
+      } finally {
+        pauseLock.unlock();
+        timeUpdate.addAndGet(System.nanoTime() - start);
+      }
+    }
+
+    /**
+     * Request a wakeup for any threads stalled in {@link #pauseNanos}.
+     */
+    public void wakeup() {
+      pauseLock.lock();
+      try {
+        pausing.signalAll();
+      } finally {
+        pauseLock.unlock();
+      }
+    }
+
+    /** Returns pause reasons and associated times in nanoseconds. */
+    public Map<PauseReason,Long> getPauseTimes() {
+      Set<Entry<PauseReason,AtomicLong>> entries = pauseTimesNS.entrySet();
+      return entries.stream()
+          .collect(Collectors.toMap(
+              (e) -> e.getKey(),
+              (e) -> e.getValue().get()));
+    }
+
+    final void setMergeThread(Thread owner) {
+      assert this.owner == null;
+      this.owner = owner;
+    }
+  }
 
   /** OneMerge provides the information necessary to perform
    *  an individual primitive merge operation, resulting in
@@ -64,7 +190,6 @@ public abstract class MergePolicy {
    *
    * @lucene.experimental */
   public static class OneMerge {
-
     SegmentCommitInfo info;         // used by IndexWriter
     boolean registerDone;           // used by IndexWriter
     long mergeGen;                  // used by IndexWriter
@@ -82,8 +207,10 @@ public abstract class MergePolicy {
     /** Segments to be merged. */
     public final List<SegmentCommitInfo> segments;
 
-    /** A private {@link RateLimiter} for this merge, used to rate limit writes and abort. */
-    public final MergeRateLimiter rateLimiter;
+    /**
+     * Control used to pause/stop/resume the merge thread. 
+     */
+    private final OneMergeProgress mergeProgress;
 
     volatile long mergeStartNS = -1;
 
@@ -106,9 +233,17 @@ public abstract class MergePolicy {
       }
       totalMaxDoc = count;
 
-      rateLimiter = new MergeRateLimiter(this);
+      mergeProgress = new OneMergeProgress();
     }
 
+    /** 
+     * Called by {@link IndexWriter} after the merge started and from the
+     * thread that will be executing the merge.
+     */
+    public void mergeInit() throws IOException {
+      mergeProgress.setMergeThread(Thread.currentThread());
+    }
+    
     /** Called by {@link IndexWriter} after the merge is done and all readers have been closed. */
     public void mergeFinished() throws IOException {
     }
@@ -163,7 +298,7 @@ public abstract class MergePolicy {
       if (maxNumSegments != -1) {
         b.append(" [maxNumSegments=" + maxNumSegments + "]");
       }
-      if (rateLimiter.getAbort()) {
+      if (isAborted()) {
         b.append(" [ABORTED]");
       }
       return b.toString();
@@ -194,7 +329,32 @@ public abstract class MergePolicy {
     /** Return {@link MergeInfo} describing this merge. */
     public MergeInfo getStoreMergeInfo() {
       return new MergeInfo(totalMaxDoc, estimatedMergeBytes, isExternal, maxNumSegments);
-    }    
+    }
+
+    /** Returns true if this merge was or should be aborted. */
+    public boolean isAborted() {
+      return mergeProgress.isAborted();
+    }
+
+    /** Marks this merge as aborted. The merge thread should terminate at the soonest possible moment. */
+    public void setAborted() {
+      this.mergeProgress.abort();
+    }
+
+    /** Checks if merge has been aborted and throws a merge exception if so. */
+    public void checkAborted() throws MergeAbortedException {
+      if (isAborted()) {
+        throw new MergePolicy.MergeAbortedException("merge is aborted: " + segString());
+      }
+    }
+
+    /**
+     * Returns a {@link OneMergeProgress} instance for this merge, which provides
+     * statistics of the merge threads (run time vs. sleep time) if merging is throttled.
+     */
+    public OneMergeProgress getMergeProgress() {
+      return mergeProgress;
+    }
   }
 
   /**
@@ -222,8 +382,7 @@ public abstract class MergePolicy {
       merges.add(merge);
     }
 
-    /** Returns a description of the merges in this
-    *  specification. */
+    /** Returns a description of the merges in this specification. */
     public String segString(Directory dir) {
       StringBuilder b = new StringBuilder();
       b.append("MergeSpec:\n");
@@ -235,8 +394,7 @@ public abstract class MergePolicy {
     }
   }
 
-  /** Exception thrown if there are any problems while
-   *  executing a merge. */
+  /** Exception thrown if there are any problems while executing a merge. */
   public static class MergeException extends RuntimeException {
     private Directory dir;
 
@@ -259,9 +417,9 @@ public abstract class MergePolicy {
     }
   }
 
-  /** Thrown when a merge was explicity aborted because
+  /** Thrown when a merge was explicitly aborted because
    *  {@link IndexWriter#abortMerges} was called.  Normally
-   *  this exception is privately caught and suppresed by
+   *  this exception is privately caught and suppressed by
    *  {@link IndexWriter}. */
   public static class MergeAbortedException extends IOException {
     /** Create a {@link MergeAbortedException}. */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9540bc37/lucene/core/src/java/org/apache/lucene/index/MergeRateLimiter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/MergeRateLimiter.java b/lucene/core/src/java/org/apache/lucene/index/MergeRateLimiter.java
index d04c2d2..e5361d5 100644
--- a/lucene/core/src/java/org/apache/lucene/index/MergeRateLimiter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/MergeRateLimiter.java
@@ -20,118 +20,107 @@ package org.apache.lucene.index;
 import org.apache.lucene.store.RateLimiter;
 import org.apache.lucene.util.ThreadInterruptedException;
 
-import static org.apache.lucene.store.RateLimiter.SimpleRateLimiter;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.lucene.index.MergePolicy.OneMergeProgress;
+import org.apache.lucene.index.MergePolicy.OneMergeProgress.PauseReason;
 
 /** This is the {@link RateLimiter} that {@link IndexWriter} assigns to each running merge, to 
  *  give {@link MergeScheduler}s ionice like control.
  *
- *  This is similar to {@link SimpleRateLimiter}, except it's merge-private,
- *  it will wake up if its rate changes while it's paused, it tracks how
- *  much time it spent stopped and paused, and it supports aborting.
- *
  *  @lucene.internal */
 
 public class MergeRateLimiter extends RateLimiter {
 
   private final static int MIN_PAUSE_CHECK_MSEC = 25;
-  volatile long totalBytesWritten;
+  
+  private final static long MIN_PAUSE_NS = TimeUnit.MILLISECONDS.toNanos(2);
+  private final static long MAX_PAUSE_NS = TimeUnit.MILLISECONDS.toNanos(250);
+
+  private volatile double mbPerSec;
+  private volatile long minPauseCheckBytes;
 
-  double mbPerSec;
   private long lastNS;
-  private long minPauseCheckBytes;
-  private boolean abort;
-  long totalPausedNS;
-  long totalStoppedNS;
-  final MergePolicy.OneMerge merge;
 
-  /** Returned by {@link #maybePause}. */
-  private static enum PauseResult {NO, STOPPED, PAUSED};
+  private AtomicLong totalBytesWritten = new AtomicLong();
 
-  /** Sole constructor. */
-  public MergeRateLimiter(MergePolicy.OneMerge merge) {
-    this.merge = merge;
+  private final OneMergeProgress mergeProgress;
 
+  /** Sole constructor. */
+  public MergeRateLimiter(OneMergeProgress mergeProgress) {
     // Initially no IO limit; use setter here so minPauseCheckBytes is set:
+    this.mergeProgress = mergeProgress;
     setMBPerSec(Double.POSITIVE_INFINITY);
   }
 
   @Override
-  public synchronized void setMBPerSec(double mbPerSec) {
-    // 0.0 is allowed: it means the merge is paused
-    if (mbPerSec < 0.0) {
-      throw new IllegalArgumentException("mbPerSec must be positive; got: " + mbPerSec);
+  public void setMBPerSec(double mbPerSec) {
+    // Synchronized to make updates to mbPerSec and minPauseCheckBytes atomic. 
+    synchronized (this) {
+      // 0.0 is allowed: it means the merge is paused
+      if (mbPerSec < 0.0) {
+        throw new IllegalArgumentException("mbPerSec must be positive; got: " + mbPerSec);
+      }
+      this.mbPerSec = mbPerSec;
+  
+      // NOTE: Double.POSITIVE_INFINITY casts to Long.MAX_VALUE
+      this.minPauseCheckBytes = Math.min(1024*1024, (long) ((MIN_PAUSE_CHECK_MSEC / 1000.0) * mbPerSec * 1024 * 1024));
+      assert minPauseCheckBytes >= 0;
     }
-    this.mbPerSec = mbPerSec;
-    // NOTE: Double.POSITIVE_INFINITY casts to Long.MAX_VALUE
-    minPauseCheckBytes = Math.min(1024*1024, (long) ((MIN_PAUSE_CHECK_MSEC / 1000.0) * mbPerSec * 1024 * 1024));
-    assert minPauseCheckBytes >= 0;
-    notify();
+
+    mergeProgress.wakeup();
   }
 
   @Override
-  public synchronized double getMBPerSec() {
+  public double getMBPerSec() {
     return mbPerSec;
   }
 
   /** Returns total bytes written by this merge. */
   public long getTotalBytesWritten() {
-    return totalBytesWritten;
+    return totalBytesWritten.get();
   }
 
   @Override
   public long pause(long bytes) throws MergePolicy.MergeAbortedException {
+    totalBytesWritten.addAndGet(bytes);
 
-    totalBytesWritten += bytes;
-
-    long startNS = System.nanoTime();
-    long curNS = startNS;
-
-    // While loop because 1) Thread.wait doesn't always sleep long
-    // enough, and 2) we wake up and check again when our rate limit
+    // While loop because we may wake up and check again when our rate limit
     // is changed while we were pausing:
-    long pausedNS = 0;
-    while (true) {
-      PauseResult result = maybePause(bytes, curNS);
-      if (result == PauseResult.NO) {
-        // Set to curNS, not targetNS, to enforce the instant rate, not
-        // the "averaaged over all history" rate:
-        lastNS = curNS;
-        break;
-      }
-      curNS = System.nanoTime();
-      long ns = curNS - startNS;
-      startNS = curNS;
-
-      // Separately track when merge was stopped vs rate limited:
-      if (result == PauseResult.STOPPED) {
-        totalStoppedNS += ns;
-      } else {
-        assert result == PauseResult.PAUSED;
-        totalPausedNS += ns;
-      }
-      pausedNS += ns;
+    long paused = 0;
+    long delta;
+    while ((delta = maybePause(bytes, System.nanoTime())) >= 0) {
+      // Keep waiting.
+      paused += delta;
     }
 
-    return pausedNS;
+    return paused;
   }
 
   /** Total NS merge was stopped. */
-  public synchronized long getTotalStoppedNS() {
-    return totalStoppedNS;
+  public long getTotalStoppedNS() {
+    return mergeProgress.getPauseTimes().get(PauseReason.STOPPED);
   } 
 
   /** Total NS merge was paused to rate limit IO. */
-  public synchronized long getTotalPausedNS() {
-    return totalPausedNS;
+  public long getTotalPausedNS() {
+    return mergeProgress.getPauseTimes().get(PauseReason.PAUSED);
   } 
 
-  /** Returns NO if no pause happened, STOPPED if pause because rate was 0.0 (merge is stopped), PAUSED if paused with a normal rate limit. */
-  private synchronized PauseResult maybePause(long bytes, long curNS) throws MergePolicy.MergeAbortedException {
-
+  /** 
+   * Returns the number of nanoseconds spent in a paused state or <code>-1</code>
+   * if no pause was applied. If the thread needs pausing, this method delegates 
+   * to the linked {@link OneMergeProgress}. 
+   */
+  private long maybePause(long bytes, long curNS) throws MergePolicy.MergeAbortedException {
     // Now is a good time to abort the merge:
-    checkAbort();
+    if (mergeProgress.isAborted()) {
+      throw new MergePolicy.MergeAbortedException("Merge aborted.");
+    }
 
-    double secondsToPause = (bytes/1024./1024.) / mbPerSec;
+    double rate = mbPerSec; // read from volatile rate once.
+    double secondsToPause = (bytes/1024./1024.) / rate;
 
     // Time we should sleep until; this is purely instantaneous
     // rate (just adds seconds onto the last time we had paused to);
@@ -140,54 +129,30 @@ public class MergeRateLimiter extends RateLimiter {
 
     long curPauseNS = targetNS - curNS;
 
-    // NOTE: except maybe on real-time JVMs, minimum realistic
-    // wait/sleep time is 1 msec; if you pass just 1 nsec the impl
-    // rounds up to 1 msec, so we don't bother unless it's > 2 msec:
-
-    if (curPauseNS <= 2000000) {
-      return PauseResult.NO;
+    // We don't bother with thread pausing if the pause is smaller than 2 msec.
+    if (curPauseNS <= MIN_PAUSE_NS) {
+      // Set to curNS, not targetNS, to enforce the instant rate, not
+      // the "averaged over all history" rate:
+      lastNS = curNS;
+      return -1;
     }
 
-    // Defensive: sleep for at most 250 msec; the loop above will call us again if we should keep sleeping:
-    if (curPauseNS > 250L*1000000) {
-      curPauseNS = 250L*1000000;
+    // Defensive: don't sleep for too long; the loop above will call us again if
+    // we should keep sleeping and the rate may be adjusted in between.
+    if (curPauseNS > MAX_PAUSE_NS) {
+      curPauseNS = MAX_PAUSE_NS;
     }
 
-    int sleepMS = (int) (curPauseNS / 1000000);
-    int sleepNS = (int) (curPauseNS % 1000000);
-
-    double rate = mbPerSec;
-
+    long start = System.nanoTime();
     try {
-      // CMS can wake us up here if it changes our target rate:
-      wait(sleepMS, sleepNS);
+      mergeProgress.pauseNanos(
+          curPauseNS, 
+          rate == 0.0 ? PauseReason.STOPPED : PauseReason.PAUSED,
+          () -> rate == mbPerSec);
     } catch (InterruptedException ie) {
       throw new ThreadInterruptedException(ie);
     }
-
-    if (rate == 0.0) {
-      return PauseResult.STOPPED;
-    } else {
-      return PauseResult.PAUSED;
-    }
-  }
-
-  /** Throws {@link MergePolicy.MergeAbortedException} if this merge was aborted. */
-  public synchronized void checkAbort() throws MergePolicy.MergeAbortedException {
-    if (abort) {
-      throw new MergePolicy.MergeAbortedException("merge is aborted: " + merge.segString());
-    }
-  }
-
-  /** Mark this merge aborted. */
-  public synchronized void setAbort() {
-    abort = true;
-    notify();
-  }
-
-  /** Returns true if this merge was aborted. */
-  public synchronized boolean getAbort() {
-    return abort;
+    return System.nanoTime() - start;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9540bc37/lucene/core/src/java/org/apache/lucene/index/MergeScheduler.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/MergeScheduler.java b/lucene/core/src/java/org/apache/lucene/index/MergeScheduler.java
index 65af45b..66d0870 100644
--- a/lucene/core/src/java/org/apache/lucene/index/MergeScheduler.java
+++ b/lucene/core/src/java/org/apache/lucene/index/MergeScheduler.java
@@ -20,6 +20,9 @@ package org.apache.lucene.index;
 import java.io.Closeable;
 import java.io.IOException;
 
+import org.apache.lucene.index.MergePolicy.OneMerge;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.RateLimitedIndexOutput;
 import org.apache.lucene.util.InfoStream;
 
 /** <p>Expert: {@link IndexWriter} uses an instance
@@ -42,6 +45,15 @@ public abstract class MergeScheduler implements Closeable {
    * */
   public abstract void merge(IndexWriter writer, MergeTrigger trigger, boolean newMergesFound) throws IOException;
 
+  /** 
+   * Wraps the incoming {@link Directory} so that we can merge-throttle it
+   * using {@link RateLimitedIndexOutput}. 
+   */
+  public Directory wrapForMerge(OneMerge merge, Directory in) {
+    // A no-op by default.
+    return in;
+  }
+
   /** Close this MergeScheduler. */
   @Override
   public abstract void close() throws IOException;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9540bc37/lucene/core/src/java/org/apache/lucene/index/NoMergeScheduler.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/NoMergeScheduler.java b/lucene/core/src/java/org/apache/lucene/index/NoMergeScheduler.java
index 1630653..e4c0136 100644
--- a/lucene/core/src/java/org/apache/lucene/index/NoMergeScheduler.java
+++ b/lucene/core/src/java/org/apache/lucene/index/NoMergeScheduler.java
@@ -16,6 +16,8 @@
  */
 package org.apache.lucene.index;
 
+import org.apache.lucene.index.MergePolicy.OneMerge;
+import org.apache.lucene.store.Directory;
 
 /**
  * A {@link MergeScheduler} which never executes any merges. It is also a
@@ -41,6 +43,11 @@ public final class NoMergeScheduler extends MergeScheduler {
 
   @Override
   public void merge(IndexWriter writer, MergeTrigger trigger, boolean newMergesFound) {}
+  
+  @Override
+  public Directory wrapForMerge(OneMerge merge, Directory in) {
+    return in;
+  }
 
   @Override
   public MergeScheduler clone() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9540bc37/lucene/core/src/test/org/apache/lucene/index/TestMergeRateLimiter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestMergeRateLimiter.java b/lucene/core/src/test/org/apache/lucene/index/TestMergeRateLimiter.java
index ef922bb..723cfbc 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestMergeRateLimiter.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestMergeRateLimiter.java
@@ -27,8 +27,8 @@ public class TestMergeRateLimiter extends LuceneTestCase {
     RandomIndexWriter w = new RandomIndexWriter(random(), dir);
     w.addDocument(new Document());
     w.close();
-    MergePolicy.OneMerge merge = new MergePolicy.OneMerge(SegmentInfos.readLatestCommit(dir).asList());
-    MergeRateLimiter rateLimiter = new MergeRateLimiter(merge);
+
+    MergeRateLimiter rateLimiter = new MergeRateLimiter(new MergePolicy.OneMergeProgress());
     assertEquals(Double.POSITIVE_INFINITY, rateLimiter.getMBPerSec(), 0.0);
     assertTrue(rateLimiter.getMinPauseCheckBytes() > 0);
     dir.close();


[17/43] lucene-solr:feature/autoscaling: SOLR-10248: Merge SolrTestCaseJ4's SolrIndexSearcher tracking into the ObjectReleaseTracker.

Posted by sh...@apache.org.
SOLR-10248: Merge SolrTestCaseJ4's SolrIndexSearcher tracking into the ObjectReleaseTracker.


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

Branch: refs/heads/feature/autoscaling
Commit: e35881a63aa9de72cf5c539396266e0d0e676956
Parents: 6a6e303
Author: Mark Miller <ma...@apache.org>
Authored: Wed Mar 8 11:30:08 2017 -0500
Committer: Mark Miller <ma...@apache.org>
Committed: Wed Mar 8 11:44:23 2017 -0500

----------------------------------------------------------------------
 .../apache/solr/search/SolrIndexSearcher.java   |  3 +
 .../java/org/apache/solr/SolrTestCaseJ4.java    | 71 +++++---------------
 2 files changed, 19 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e35881a6/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
index 521324a..a7ee433 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
@@ -105,6 +105,7 @@ import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.ObjectReleaseTracker;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.core.DirectoryFactory;
 import org.apache.solr.core.DirectoryFactory.DirContext;
@@ -391,6 +392,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
     readerStats = snapStatistics(reader);
     // do this at the end since an exception in the constructor means we won't close
     numOpens.incrementAndGet();
+    assert ObjectReleaseTracker.track(this);
   }
 
   /*
@@ -539,6 +541,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
 
     // do this at the end so it only gets done if there are no exceptions
     numCloses.incrementAndGet();
+    assert ObjectReleaseTracker.release(this);
   }
 
   /** Direct access to the IndexSchema for use with this searcher */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e35881a6/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 a8c93d6..825e7c7 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
@@ -297,17 +297,10 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
       if (suiteFailureMarker.wasSuccessful()) {
         // if the tests passed, make sure everything was closed / released
         if (!RandomizedContext.current().getTargetClass().isAnnotationPresent(SuppressObjectReleaseTracker.class)) {
-          endTrackingSearchers(120, false);
-          String orr = clearObjectTrackerAndCheckEmpty(120);
+          String orr = clearObjectTrackerAndCheckEmpty(20, false);
           assertNull(orr, orr);
         } else {
-          endTrackingSearchers(15, false);
-          String orr = ObjectReleaseTracker.checkEmpty();
-          if (orr != null) {
-            log.warn(
-                "Some resources were not closed, shutdown, or released. This has been ignored due to the SuppressObjectReleaseTracker annotation, trying to close them now.");
-            ObjectReleaseTracker.tryClose();
-          }
+          clearObjectTrackerAndCheckEmpty(20, true);
         }
       }
       resetFactory();
@@ -341,6 +334,13 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
    * @return null if ok else error message
    */
   public static String clearObjectTrackerAndCheckEmpty(int waitSeconds) {
+    return clearObjectTrackerAndCheckEmpty(waitSeconds, false);
+  }
+  
+  /**
+   * @return null if ok else error message
+   */
+  public static String clearObjectTrackerAndCheckEmpty(int waitSeconds, boolean tryClose) {
     int retries = 0;
     String result;
     do {
@@ -367,6 +367,13 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
     
     
     log.info("------------------------------------------------------- Done waiting for tracked resources to be released");
+    
+    if (tryClose && result != null && RandomizedContext.current().getTargetClass().isAnnotationPresent(SuppressObjectReleaseTracker.class)) {
+      log.warn(
+          "Some resources were not closed, shutdown, or released. This has been ignored due to the SuppressObjectReleaseTracker annotation, trying to close them now.");
+      ObjectReleaseTracker.tryClose();
+    }
+    
     ObjectReleaseTracker.clear();
     
     return result;
@@ -580,52 +587,6 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
       numOpens = numCloses = 0;
     }
   }
-
-  public static void endTrackingSearchers(int waitSeconds, boolean failTest) {
-     long endNumOpens = SolrIndexSearcher.numOpens.get();
-     long endNumCloses = SolrIndexSearcher.numCloses.get();
-
-     // wait a bit in case any ending threads have anything to release
-     int retries = 0;
-     while (endNumOpens - numOpens != endNumCloses - numCloses) {
-       if (retries++ > waitSeconds) {
-         break;
-       }
-       if (retries % 10 == 0) {
-         log.info("Waiting for all SolrIndexSearchers to be released at end of test");
-        if (retries > 10) {
-          TraceFormatting tf = new TraceFormatting();
-          Map<Thread,StackTraceElement[]> stacksMap = Thread.getAllStackTraces();
-          Set<Entry<Thread,StackTraceElement[]>> entries = stacksMap.entrySet();
-          for (Entry<Thread,StackTraceElement[]> entry : entries) {
-            String stack = tf.formatStackTrace(entry.getValue());
-            System.err.println(entry.getKey().getName() + ":\n" + stack);
-          }
-        }
-       }
-       try {
-         Thread.sleep(1000);
-       } catch (InterruptedException e) {}
-       endNumOpens = SolrIndexSearcher.numOpens.get();
-       endNumCloses = SolrIndexSearcher.numCloses.get();
-     }
-
-     log.info("------------------------------------------------------- Done waiting for all SolrIndexSearchers to be released");
-     
-     SolrIndexSearcher.numOpens.getAndSet(0);
-     SolrIndexSearcher.numCloses.getAndSet(0);
-
-     if (endNumOpens-numOpens != endNumCloses-numCloses) {
-       String msg = "ERROR: SolrIndexSearcher opens=" + (endNumOpens-numOpens) + " closes=" + (endNumCloses-numCloses);
-       log.error(msg);
-       // if it's TestReplicationHandler, ignore it. the test is broken and gets no love
-       if ("TestReplicationHandler".equals(RandomizedContext.current().getTargetClass().getSimpleName())) {
-         log.warn("TestReplicationHandler wants to fail!: " + msg);
-       } else {
-         if (failTest) fail(msg);
-       }
-     }
-  }
   
   /** Causes an exception matching the regex pattern to not be logged. */
   public static void ignoreException(String pattern) {


[11/43] lucene-solr:feature/autoscaling: LUCENE-7718: buildAndPushRelease.py script should refer to working tree instead of directory

Posted by sh...@apache.org.
LUCENE-7718: buildAndPushRelease.py script should refer to working tree instead of directory


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

Branch: refs/heads/feature/autoscaling
Commit: a6e14ec6d2d176f8363efc46b0685fda9a0942b2
Parents: 3a99339
Author: Ishan Chattopadhyaya <is...@apache.org>
Authored: Tue Feb 28 21:22:17 2017 +0530
Committer: Ishan Chattopadhyaya <is...@apache.org>
Committed: Wed Mar 8 01:49:13 2017 +0530

----------------------------------------------------------------------
 dev-tools/scripts/buildAndPushRelease.py | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a6e14ec6/dev-tools/scripts/buildAndPushRelease.py
----------------------------------------------------------------------
diff --git a/dev-tools/scripts/buildAndPushRelease.py b/dev-tools/scripts/buildAndPushRelease.py
index e34c943..d742214 100644
--- a/dev-tools/scripts/buildAndPushRelease.py
+++ b/dev-tools/scripts/buildAndPushRelease.py
@@ -59,7 +59,7 @@ def runAndSendGPGPassword(command, password):
 
 def getGitRev():
   status = os.popen('git status').read().strip()
-  if 'nothing to commit, working directory clean' not in status:
+  if 'nothing to commit, working directory clean' not in status and 'nothing to commit, working tree clean' not in status:
     raise RuntimeError('git clone is dirty:\n\n%s' % status)
   branch = os.popen('git rev-parse --abbrev-ref HEAD').read().strip()
   command = 'git log origin/%s..' % branch


[26/43] lucene-solr:feature/autoscaling: SOLR-8045: Deploy V2 API at /v2 instead of /solr/v2

Posted by sh...@apache.org.
SOLR-8045: Deploy V2 API at /v2 instead of /solr/v2


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

Branch: refs/heads/feature/autoscaling
Commit: 0fb386a864ff5b7d32af3bef3f7eeca4d009acc1
Parents: 6415d91
Author: Cao Manh Dat <da...@apache.org>
Authored: Sat Mar 11 10:30:52 2017 +0700
Committer: Cao Manh Dat <da...@apache.org>
Committed: Sat Mar 11 10:30:52 2017 +0700

----------------------------------------------------------------------
 dev-tools/scripts/smokeTestRelease.py           |  4 ++
 solr/CHANGES.txt                                |  2 +
 .../java/org/apache/solr/api/V2HttpCall.java    |  2 +-
 .../java/org/apache/solr/core/PluginBag.java    |  2 +-
 .../apache/solr/servlet/SolrDispatchFilter.java |  2 +-
 .../conf/solrconfig-managed-schema.xml          |  2 +-
 .../org/apache/solr/cloud/rule/RulesTest.java   |  2 +-
 .../apache/solr/core/TestDynamicLoading.java    |  2 +-
 .../apache/solr/core/TestSolrConfigHandler.java | 14 ++---
 .../apache/solr/handler/TestReqParamsAPI.java   |  3 ++
 .../solr/handler/V2ApiIntegrationTest.java      |  4 +-
 .../solr/rest/schema/TestBulkSchemaAPI.java     |  2 +-
 .../solr/security/BasicAuthIntegrationTest.java |  4 +-
 solr/server/etc/jetty.xml                       | 54 +++++++++++---------
 .../conf/solrconfig.xml                         |  2 +-
 .../solr/client/solrj/impl/CloudSolrClient.java |  4 +-
 .../solrj/embedded/SolrExampleJettyTest.java    |  2 +-
 17 files changed, 62 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0fb386a8/dev-tools/scripts/smokeTestRelease.py
----------------------------------------------------------------------
diff --git a/dev-tools/scripts/smokeTestRelease.py b/dev-tools/scripts/smokeTestRelease.py
index f9c3499..b0e76e9 100644
--- a/dev-tools/scripts/smokeTestRelease.py
+++ b/dev-tools/scripts/smokeTestRelease.py
@@ -855,6 +855,10 @@ def testSolrExample(unpackPath, javaPath, isSrc):
     if s.find('<result name="response" numFound="3" start="0">') == -1:
       print('FAILED: response is:\n%s' % s)
       raise RuntimeError('query on solr example instance failed')
+    s = load('http://localhost:8983/v2/cores')
+    if s.find('"responseHeader":{"status":0,"QTime":1}') == -1:
+      print('FAILED: response is:\n%s' % s)
+      raise RuntimeError('query api v2 on solr example instance failed')
   finally:
     # Stop server:
     print('      stop server using: bin/solr stop -p 8983')

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0fb386a8/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 7285e4f..b164405 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -164,6 +164,8 @@ New Features
 
 * SOLR-9986: Implement DatePointField (Cao Manh Dat, Tom�s Fern�ndez L�bbe)
 
+* SOLR-8045: Deploy V2 API at /v2 instead of /solr/v2 (Cao Manh Dat, Noble Paul)
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0fb386a8/solr/core/src/java/org/apache/solr/api/V2HttpCall.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/api/V2HttpCall.java b/solr/core/src/java/org/apache/solr/api/V2HttpCall.java
index c996b25..fb4aa56 100644
--- a/solr/core/src/java/org/apache/solr/api/V2HttpCall.java
+++ b/solr/core/src/java/org/apache/solr/api/V2HttpCall.java
@@ -75,7 +75,7 @@ public class V2HttpCall extends HttpSolrCall {
 
   protected void init() throws Exception {
     String path = this.path;
-    String fullPath = path = path.substring(3);//strip off '/v2'
+    String fullPath = path = path.substring(7);//strip off '/____v2'
     try {
       pieces = getPathSegments(path);
       if (pieces.size() == 0) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0fb386a8/solr/core/src/java/org/apache/solr/core/PluginBag.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/PluginBag.java b/solr/core/src/java/org/apache/solr/core/PluginBag.java
index ad8bdec..65978f3 100644
--- a/solr/core/src/java/org/apache/solr/core/PluginBag.java
+++ b/solr/core/src/java/org/apache/solr/core/PluginBag.java
@@ -194,7 +194,7 @@ public class PluginBag<T> implements AutoCloseable {
       String registerAt = plugin.pluginInfo.attributes.get("registerPath");
       if (registerAt != null) {
         List<String> strs = StrUtils.splitSmart(registerAt, ',');
-        disableHandler = !strs.contains("/");
+        disableHandler = !strs.contains("/solr");
         registerApi = strs.contains("/v2");
       }
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0fb386a8/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 ce65069..ff0db9b 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
@@ -388,7 +388,7 @@ public class SolrDispatchFilter extends BaseSolrFilter {
       path += request.getPathInfo();
     }
 
-    if (isV2Enabled && (path.startsWith("/v2/") || path.equals("/v2"))) {
+    if (isV2Enabled && (path.startsWith("/____v2/") || path.equals("/____v2"))) {
       return new V2HttpCall(this, cores, request, response, false);
     } else {
       return new HttpSolrCall(this, cores, request, response, retry);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0fb386a8/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml
index 31bbbb3..abd4fbe 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml
@@ -40,7 +40,7 @@
   </requestHandler>
 
 
-  <requestHandler name="/dump" class="DumpRequestHandler" initParams="a" registerPath="/,/v2">
+  <requestHandler name="/dump" class="DumpRequestHandler" initParams="a" registerPath="/solr,/v2">
     <lst name="defaults">
       <str name="a">${my.custom.variable.a:A}</str>
       <str name="b">${my.custom.variable.b:B}</str>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0fb386a8/solr/core/src/test/org/apache/solr/cloud/rule/RulesTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/rule/RulesTest.java b/solr/core/src/test/org/apache/solr/cloud/rule/RulesTest.java
index 13649e1..d4a72bf 100644
--- a/solr/core/src/test/org/apache/solr/cloud/rule/RulesTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/rule/RulesTest.java
@@ -168,7 +168,7 @@ public class RulesTest extends SolrCloudTestCase {
   public void testInvokeApi() throws Exception {
     JettySolrRunner jetty = cluster.getRandomJetty(random());
     try (SolrClient client = getHttpSolrClient(jetty.getBaseUrl().toString())) {
-      GenericSolrRequest req =  new GenericSolrRequest(GET, "/v2/node/invoke", new ModifiableSolrParams()
+      GenericSolrRequest req =  new GenericSolrRequest(GET, "/____v2/node/invoke", new ModifiableSolrParams()
           .add("class", ImplicitSnitch.class.getName())
           .add("cores", "1")
           .add("freedisk", "1")

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0fb386a8/solr/core/src/test/org/apache/solr/core/TestDynamicLoading.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/TestDynamicLoading.java b/solr/core/src/test/org/apache/solr/core/TestDynamicLoading.java
index 8479ae4..306b4b2 100644
--- a/solr/core/src/test/org/apache/solr/core/TestDynamicLoading.java
+++ b/solr/core/src/test/org/apache/solr/core/TestDynamicLoading.java
@@ -88,7 +88,7 @@ public class TestDynamicLoading extends AbstractFullDistribZkTestBase {
 
 
     payload = "{\n" +
-        "'create-requesthandler' : { 'name' : '/test1', 'class': 'org.apache.solr.core.BlobStoreTestRequestHandler' ,registerPath: '/,/v2',  'runtimeLib' : true }\n" +
+        "'create-requesthandler' : { 'name' : '/test1', 'class': 'org.apache.solr.core.BlobStoreTestRequestHandler' ,registerPath: '/solr,/v2',  'runtimeLib' : true }\n" +
         "}";
 
     client = restTestHarnesses.get(random().nextInt(restTestHarnesses.size()));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0fb386a8/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java b/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java
index 3f85a79..ec81c25 100644
--- a/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java
+++ b/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java
@@ -86,7 +86,7 @@ public class TestSolrConfigHandler extends RestTestBase {
         "/solr", true, extraServlets);
     if (random().nextBoolean()) {
       log.info("These tests are run with V2 API");
-      restTestHarness.setServerProvider(() -> jetty.getBaseUrl().toString() + "/v2/cores/" + DEFAULT_TEST_CORENAME);
+      restTestHarness.setServerProvider(() -> jetty.getBaseUrl().toString() + "/____v2/cores/" + DEFAULT_TEST_CORENAME);
     }
   }
 
@@ -201,7 +201,7 @@ public class TestSolrConfigHandler extends RestTestBase {
         10);
 
     payload = "{\n" +
-        "'update-requesthandler' : { 'name' : '/x', 'class': 'org.apache.solr.handler.DumpRequestHandler' ,registerPath :'/,/v2', " +
+        "'update-requesthandler' : { 'name' : '/x', 'class': 'org.apache.solr.handler.DumpRequestHandler' ,registerPath :'/solr,/v2', " +
         " 'startup' : 'lazy' , 'a':'b' , 'defaults': {'def_a':'def A val', 'multival':['a','b','c']}}\n" +
         "}";
     runConfigCommand(writeHarness, "/config?wt=json", payload);
@@ -442,7 +442,7 @@ public class TestSolrConfigHandler extends RestTestBase {
     payload = "{\n" +
         "    'add-requesthandler': {\n" +
         "        name : '/dump100',\n" +
-        "       registerPath :'/,/v2',"+
+        "       registerPath :'/solr,/v2',"+
     "        class : 'org.apache.solr.handler.DumpRequestHandler'," +
         "        suggester: [{name: s1,lookupImpl: FuzzyLookupFactory, dictionaryImpl : DocumentDictionaryFactory}," +
         "                    {name: s2,lookupImpl: FuzzyLookupFactory , dictionaryImpl : DocumentExpressionDictionaryFactory}]" +
@@ -467,7 +467,7 @@ public class TestSolrConfigHandler extends RestTestBase {
     payload = "{\n" +
         "'add-requesthandler' : { 'name' : '/dump101', 'class': " +
         "'" + CacheTest.class.getName() + "', " +
-        "    registerPath :'/,/v2'"+
+        "    registerPath :'/solr,/v2'"+
         ", 'startup' : 'lazy'}\n" +
         "}";
     runConfigCommand(writeHarness, "/config?wt=json", payload);
@@ -589,7 +589,7 @@ public class TestSolrConfigHandler extends RestTestBase {
         10);
 
     payload = "{\n" +
-        "'create-requesthandler' : { 'name' : '/d', registerPath :'/,/v2' , 'class': 'org.apache.solr.handler.DumpRequestHandler' }\n" +
+        "'create-requesthandler' : { 'name' : '/d', registerPath :'/solr,/v2' , 'class': 'org.apache.solr.handler.DumpRequestHandler' }\n" +
         "}";
 
     TestSolrConfigHandler.runConfigCommand(harness, "/config?wt=json", payload);
@@ -619,7 +619,7 @@ public class TestSolrConfigHandler extends RestTestBase {
         5);
 
     payload = "{\n" +
-        "'create-requesthandler' : { 'name' : '/dump1', registerPath :'/,/v2' , 'class': 'org.apache.solr.handler.DumpRequestHandler', 'useParams':'x' }\n" +
+        "'create-requesthandler' : { 'name' : '/dump1', registerPath :'/solr,/v2' , 'class': 'org.apache.solr.handler.DumpRequestHandler', 'useParams':'x' }\n" +
         "}";
 
     TestSolrConfigHandler.runConfigCommand(harness, "/config?wt=json", payload);
@@ -794,7 +794,7 @@ public class TestSolrConfigHandler extends RestTestBase {
         "org.apache.solr.handler.DumpRequestHandler",
         10);
     RESTfulServerProvider oldProvider = restTestHarness.getServerProvider();
-    restTestHarness.setServerProvider(() -> jetty.getBaseUrl().toString() + "/v2/cores/" + DEFAULT_TEST_CORENAME);
+    restTestHarness.setServerProvider(() -> jetty.getBaseUrl().toString() + "/____v2/cores/" + DEFAULT_TEST_CORENAME);
 
     Map rsp = TestSolrConfigHandler.testForResponseElement(
         harness,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0fb386a8/solr/core/src/test/org/apache/solr/handler/TestReqParamsAPI.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/TestReqParamsAPI.java b/solr/core/src/test/org/apache/solr/handler/TestReqParamsAPI.java
index 3912011..de4a27a 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestReqParamsAPI.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestReqParamsAPI.java
@@ -50,6 +50,9 @@ public class TestReqParamsAPI extends SolrCloudTestCase {
   private void setupHarnesses() {
     for (final JettySolrRunner jettySolrRunner : cluster.getJettySolrRunners()) {
       RestTestHarness harness = new RestTestHarness(() -> jettySolrRunner.getBaseUrl().toString() + "/" + COLL_NAME);
+      if (true) {
+        harness.setServerProvider(() -> jettySolrRunner.getBaseUrl().toString() + "/____v2/c/" + COLL_NAME);
+      }
       restTestHarnesses.add(harness);
     }
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0fb386a8/solr/core/src/test/org/apache/solr/handler/V2ApiIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/V2ApiIntegrationTest.java b/solr/core/src/test/org/apache/solr/handler/V2ApiIntegrationTest.java
index 1af5d93..4eb3de2 100644
--- a/solr/core/src/test/org/apache/solr/handler/V2ApiIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/V2ApiIntegrationTest.java
@@ -86,10 +86,10 @@ public class V2ApiIntegrationTest extends SolrCloudTestCase {
   private void testApis() throws Exception {
     RestTestHarness restHarness = restTestHarnesses.get(0);
     ServerProvider serverProvider = (ServerProvider) restHarness.getServerProvider();
-    serverProvider.baseurl = serverProvider.jettySolrRunner.getBaseUrl()+"/v2/c/"+ COLL_NAME;
+    serverProvider.baseurl = serverProvider.jettySolrRunner.getBaseUrl()+"/____v2/c/"+ COLL_NAME;
     Map result = TestSolrConfigHandler.getRespMap("/get/_introspect", restHarness);
     assertEquals("/c/collection1/get", Utils.getObjectByPath(result, true, "/spec[0]/url/paths[0]"));
-    serverProvider.baseurl = serverProvider.jettySolrRunner.getBaseUrl()+"/v2/collections/"+ COLL_NAME;
+    serverProvider.baseurl = serverProvider.jettySolrRunner.getBaseUrl()+"/____v2/collections/"+ COLL_NAME;
     result = TestSolrConfigHandler.getRespMap("/get/_introspect", restHarness);
     assertEquals("/collections/collection1/get", Utils.getObjectByPath(result, true, "/spec[0]/url/paths[0]"));
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0fb386a8/solr/core/src/test/org/apache/solr/rest/schema/TestBulkSchemaAPI.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/rest/schema/TestBulkSchemaAPI.java b/solr/core/src/test/org/apache/solr/rest/schema/TestBulkSchemaAPI.java
index ea8fd7b..e2dc2bf 100644
--- a/solr/core/src/test/org/apache/solr/rest/schema/TestBulkSchemaAPI.java
+++ b/solr/core/src/test/org/apache/solr/rest/schema/TestBulkSchemaAPI.java
@@ -69,7 +69,7 @@ public class TestBulkSchemaAPI extends RestTestBase {
       restTestHarness.setServerProvider(new RESTfulServerProvider() {
         @Override
         public String getBaseURL() {
-          return jetty.getBaseUrl().toString() + "/v2/cores/" + DEFAULT_TEST_CORENAME;
+          return jetty.getBaseUrl().toString() + "/____v2/cores/" + DEFAULT_TEST_CORENAME;
         }
       });
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0fb386a8/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java b/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
index 397f4e8..5231dd8 100644
--- a/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
@@ -86,8 +86,8 @@ public class BasicAuthIntegrationTest extends SolrCloudTestCase {
     String authcPrefix = "/admin/authentication";
     String authzPrefix = "/admin/authorization";
     if(random().nextBoolean()){
-      authcPrefix = "/v2/cluster/security/authentication";
-      authzPrefix = "/v2/cluster/security/authorization";
+      authcPrefix = "/____v2/cluster/security/authentication";
+      authzPrefix = "/____v2/cluster/security/authorization";
     }
 
     NamedList<Object> rsp;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0fb386a8/solr/server/etc/jetty.xml
----------------------------------------------------------------------
diff --git a/solr/server/etc/jetty.xml b/solr/server/etc/jetty.xml
index 8cb8223..b512c51 100644
--- a/solr/server/etc/jetty.xml
+++ b/solr/server/etc/jetty.xml
@@ -97,35 +97,43 @@
           </New>
         </Arg>
       </Call>
+       <Call name="addRule">
+         <Arg>
+           <New class="org.eclipse.jetty.rewrite.handler.RewritePatternRule">
+             <Set name="pattern">/v2/*</Set>
+             <Set name="replacement">/solr/____v2</Set>
+           </New>
+         </Arg>
+       </Call>
+       <Set name="handler">
+         <New id="Handlers" class="org.eclipse.jetty.server.handler.HandlerCollection">
+           <Set name="handlers">
+             <Array type="org.eclipse.jetty.server.Handler">
+               <Item>
+                 <New id="Contexts" class="org.eclipse.jetty.server.handler.ContextHandlerCollection"/>
+               </Item>
+               <Item>
+                 <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"/>
+               </Item>
+             </Array>
+           </Set>
+         </New>
+       </Set>
      </New>
 
     <!-- =========================================================== -->
     <!-- Set handler Collection Structure                            -->
     <!-- =========================================================== -->
     <Set name="handler">
-      <New id="Handlers" class="org.eclipse.jetty.server.handler.HandlerCollection">
-        <Set name="handlers">
-         <Array type="org.eclipse.jetty.server.Handler">
-           <Item>
-             <Ref id="RewriteHandler"/>
-           </Item>
-           <Item>
-             <New id="Contexts" class="org.eclipse.jetty.server.handler.ContextHandlerCollection"/>
-           </Item>
-           <Item>
-             <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"/>
-           </Item>
-         </Array>
-        </Set>
-      </New>
+      <Ref id="RewriteHandler"/>
     </Set>
     
     <!-- =========================================================== -->

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0fb386a8/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 a9ddb25..3ff89c0 100644
--- a/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml
+++ b/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml
@@ -850,7 +850,7 @@
     </requestHandler>
 
   <!-- A request handler that returns indented JSON by default -->
-  <requestHandler name="/query" class="solr.SearchHandler" registerPath="/,/v2">
+  <requestHandler name="/query" class="solr.SearchHandler" registerPath="/solr,/v2">
      <lst name="defaults">
        <str name="echoParams">explicit</str>
        <str name="wt">json</str>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0fb386a8/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
index 3147d4e..d3938c8 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
@@ -1047,8 +1047,8 @@ public class CloudSolrClient extends SolrClient {
       CONFIGSETS_HANDLER_PATH,
       AUTHC_PATH,
       AUTHZ_PATH,
-      "/v2/cluster/security/authentication",
-      "/v2/cluster/security/authorization"
+      "/____v2/cluster/security/authentication",
+      "/____v2/cluster/security/authorization"
       ));
 
   /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0fb386a8/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleJettyTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleJettyTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleJettyTest.java
index b7ac7de..cb4ba50 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleJettyTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleJettyTest.java
@@ -98,7 +98,7 @@ public class SolrExampleJettyTest extends SolrExampleTests {
   private String getUri(HttpSolrClient client) {
     String baseURL = client.getBaseURL();
     return random().nextBoolean() ?
-        baseURL.replace("/collection1", "/v2/cores/collection1/update") :
+        baseURL.replace("/collection1", "/____v2/cores/collection1/update") :
         baseURL + "/update/json/docs";
   }
 }


[28/43] lucene-solr:feature/autoscaling: LUCENE-7449: fix CROSSES queries so they don't match all docs when internal nodes are equal

Posted by sh...@apache.org.
LUCENE-7449: fix CROSSES queries so they don't match all docs when internal nodes are equal


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

Branch: refs/heads/feature/autoscaling
Commit: f3ba7f41057227555992c1534a8265d37bfe7c23
Parents: a3f4896
Author: Nicholas Knize <nk...@gmail.com>
Authored: Sat Mar 11 18:40:55 2017 -0600
Committer: Nicholas Knize <nk...@gmail.com>
Committed: Sat Mar 11 18:41:18 2017 -0600

----------------------------------------------------------------------
 .../src/java/org/apache/lucene/document/RangeFieldQuery.java     | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f3ba7f41/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 41e64cf..10f10fa 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java
@@ -171,8 +171,8 @@ abstract class RangeFieldQuery extends Query {
           // if query crosses, docs need to be further scrutinized
           byte[] range = getInternalRange(values.getMinPackedValue(), values.getMaxPackedValue());
           // if the internal node is not equal and not contained by the query, all docs do not match
-          if (!Arrays.equals(ranges, range)
-              && (!target.contains(range) || queryType != QueryType.WITHIN)) {
+          if (queryType == QueryType.CROSSES || (!Arrays.equals(ranges, range)
+              && (target.contains(range) == false || queryType != QueryType.WITHIN))) {
             allDocsMatch = false;
           }
         } else {


[29/43] lucene-solr:feature/autoscaling: LUCENE-7738: Add new InetAddressRangeField for indexing and querying InetAddress ranges.

Posted by sh...@apache.org.
LUCENE-7738: Add new InetAddressRangeField for indexing and querying InetAddress ranges.


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

Branch: refs/heads/feature/autoscaling
Commit: 1745b0338e822db43f292f7ad495789b21c6634a
Parents: f3ba7f4
Author: Nicholas Knize <nk...@gmail.com>
Authored: Fri Mar 10 15:05:43 2017 -0600
Committer: Nicholas Knize <nk...@gmail.com>
Committed: Sat Mar 11 18:51:43 2017 -0600

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   3 +
 .../lucene/document/InetAddressRangeField.java  | 168 ++++++++++++++
 .../lucene/search/TestIpRangeFieldQueries.java  | 220 +++++++++++++++++++
 3 files changed, 391 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1745b033/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 9407dfa..c2fe191 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -131,6 +131,9 @@ API Changes
 
 New Features
 
+* LUCENE-7738: Add new InetAddressRangeField for indexing and querying
+  InetAddress ranges. (Nick Knize)
+
 * LUCENE-7449: Add CROSSES relation support to RangeFieldQuery. (Nick Knize)
 
 * LUCENE-7623: Add FunctionScoreQuery and FunctionMatchQuery (Alan Woodward,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1745b033/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressRangeField.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressRangeField.java b/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressRangeField.java
new file mode 100644
index 0000000..c6ebc83
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressRangeField.java
@@ -0,0 +1,168 @@
+/*
+ * 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.document;
+
+import java.net.InetAddress;
+
+import org.apache.lucene.document.RangeFieldQuery.QueryType;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.StringHelper;
+
+/**
+ * An indexed InetAddress Range Field
+ * <p>
+ * This field indexes an {@code InetAddress} range defined as a min/max pairs. It is single
+ * dimension only (indexed as two 16 byte paired values).
+ * <p>
+ * Multiple values are supported.
+ *
+ * <p>
+ * This field defines the following static factory methods for common search operations over Ip Ranges
+ * <ul>
+ *   <li>{@link #newIntersectsQuery newIntersectsQuery()} matches ip ranges that intersect the defined search range.
+ *   <li>{@link #newWithinQuery newWithinQuery()} matches ip ranges that are within the defined search range.
+ *   <li>{@link #newContainsQuery newContainsQuery()} matches ip ranges that contain the defined search range.
+ *   <li>{@link #newCrossesQuery newCrossesQuery()} matches ip ranges that cross the defined search range
+ * </ul>
+ */
+public class InetAddressRangeField extends Field {
+  /** The number of bytes per dimension : sync w/ {@code InetAddressPoint} */
+  public static final int BYTES = InetAddressPoint.BYTES;
+
+  private static final FieldType TYPE;
+  static {
+    TYPE = new FieldType();
+    TYPE.setDimensions(2, BYTES);
+    TYPE.freeze();
+  }
+
+  /**
+   * Create a new InetAddressRangeField from min/max value
+   * @param name field name. must not be null.
+   * @param min range min value; defined as an {@code InetAddress}
+   * @param max range max value; defined as an {@code InetAddress}
+   */
+  public InetAddressRangeField(String name, final InetAddress min, final InetAddress max) {
+    super(name, TYPE);
+    setRangeValues(min, max);
+  }
+
+  /**
+   * Change (or set) the min/max values of the field.
+   * @param min range min value; defined as an {@code InetAddress}
+   * @param max range max value; defined as an {@code InetAddress}
+   */
+  public void setRangeValues(InetAddress min, InetAddress max) {
+    if (StringHelper.compare(BYTES, min.getAddress(), 0, max.getAddress(), 0) > 0) {
+      throw new IllegalArgumentException("min value cannot be greater than max value for range field (name=" + name + ")");
+    }
+    final byte[] bytes;
+    if (fieldsData == null) {
+      bytes = new byte[BYTES*2];
+      fieldsData = new BytesRef(bytes);
+    } else {
+      bytes = ((BytesRef)fieldsData).bytes;
+    }
+    encode(min, max, bytes);
+  }
+
+  /** encode the min/max range into the provided byte array */
+  private static void encode(final InetAddress min, final InetAddress max, final byte[] bytes) {
+    System.arraycopy(InetAddressPoint.encode(min), 0, bytes, 0, BYTES);
+    System.arraycopy(InetAddressPoint.encode(max), 0, bytes, BYTES, BYTES);
+  }
+
+  /** encode the min/max range and return the byte array */
+  private static byte[] encode(InetAddress min, InetAddress max) {
+    byte[] b = new byte[BYTES*2];
+    encode(min, max, b);
+    return b;
+  }
+
+  /**
+   * Create a query for matching indexed ip ranges that {@code INTERSECT} the defined range.
+   * @param field field name. must not be null.
+   * @param min range min value; provided as an {@code InetAddress}
+   * @param max range max value; provided as an {@code InetAddress}
+   * @return query for matching intersecting ranges (overlap, within, crosses, or contains)
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newIntersectsQuery(String field, final InetAddress min, final InetAddress max) {
+    return newRelationQuery(field, min, max, QueryType.INTERSECTS);
+  }
+
+  /**
+   * Create a query for matching indexed ip ranges that {@code CONTAINS} the defined range.
+   * @param field field name. must not be null.
+   * @param min range min value; provided as an {@code InetAddress}
+   * @param max range max value; provided as an {@code InetAddress}
+   * @return query for matching intersecting ranges (overlap, within, crosses, or contains)
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newContainsQuery(String field, final InetAddress min, final InetAddress max) {
+    return newRelationQuery(field, min, max, QueryType.CONTAINS);
+  }
+
+  /**
+   * Create a query for matching indexed ip ranges that are {@code WITHIN} the defined range.
+   * @param field field name. must not be null.
+   * @param min range min value; provided as an {@code InetAddress}
+   * @param max range max value; provided as an {@code InetAddress}
+   * @return query for matching intersecting ranges (overlap, within, crosses, or contains)
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newWithinQuery(String field, final InetAddress min, final InetAddress max) {
+    return newRelationQuery(field, min, max, QueryType.WITHIN);
+  }
+
+  /**
+   * Create a query for matching indexed ip ranges that {@code CROSS} the defined range.
+   * @param field field name. must not be null.
+   * @param min range min value; provided as an {@code InetAddress}
+   * @param max range max value; provided as an {@code InetAddress}
+   * @return query for matching intersecting ranges (overlap, within, crosses, or contains)
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newCrossesQuery(String field, final InetAddress min, final InetAddress max) {
+    return newRelationQuery(field, min, max, QueryType.CROSSES);
+  }
+
+  /** helper method for creating the desired relational query */
+  private static Query newRelationQuery(String field, final InetAddress min, final InetAddress max, QueryType relation) {
+    return new RangeFieldQuery(field, encode(min, max), 1, relation) {
+      @Override
+      protected String toString(byte[] ranges, int dimension) {
+        return InetAddressRangeField.toString(ranges, dimension);
+      }
+    };
+  }
+
+  /**
+   * Returns the String representation for the range at the given dimension
+   * @param ranges the encoded ranges, never null
+   * @param dimension the dimension of interest (not used for this field)
+   * @return The string representation for the range at the provided dimension
+   */
+  private static String toString(byte[] ranges, int dimension) {
+    byte[] min = new byte[BYTES];
+    System.arraycopy(ranges, 0, min, 0, BYTES);
+    byte[] max = new byte[BYTES];
+    System.arraycopy(ranges, BYTES, max, 0, BYTES);
+    return "[" + InetAddressPoint.decode(min) + " : " + InetAddressPoint.decode(max) + "]";
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1745b033/lucene/sandbox/src/test/org/apache/lucene/search/TestIpRangeFieldQueries.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/TestIpRangeFieldQueries.java b/lucene/sandbox/src/test/org/apache/lucene/search/TestIpRangeFieldQueries.java
new file mode 100644
index 0000000..1563584
--- /dev/null
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/TestIpRangeFieldQueries.java
@@ -0,0 +1,220 @@
+/*
+ * 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.net.InetAddress;
+import java.net.UnknownHostException;
+
+import org.apache.lucene.document.InetAddressRangeField;
+import org.apache.lucene.util.StringHelper;
+
+/**
+ * Random testing for {@link InetAddressRangeField}
+ */
+public class TestIpRangeFieldQueries extends BaseRangeFieldQueryTestCase {
+  private static final String FIELD_NAME = "ipRangeField";
+
+  private IPVersion ipVersion;
+
+  private enum IPVersion {IPv4, IPv6}
+
+  @Override
+  protected Range nextRange(int dimensions) {
+    try {
+      InetAddress min = nextInetaddress();
+      byte[] bMin = min.getAddress();
+      InetAddress max = nextInetaddress();
+      byte[] bMax = max.getAddress();
+      if (StringHelper.compare(bMin.length, bMin, 0, bMax, 0) > 0) {
+        return new IpRange(max, min);
+      }
+      return new IpRange(min, max);
+    } catch (UnknownHostException e) {
+      e.printStackTrace();
+    }
+    return null;
+  }
+
+  /** return random IPv4 or IPv6 address */
+  private InetAddress nextInetaddress() throws UnknownHostException {
+    byte[] b;
+    switch (ipVersion) {
+      case IPv4:
+        b = new byte[4];
+        break;
+      case IPv6:
+        b = new byte[16];
+        break;
+      default:
+        throw new IllegalArgumentException("incorrect IP version: " + ipVersion);
+    }
+    random().nextBytes(b);
+    return InetAddress.getByAddress(b);
+  }
+
+  /** randomly select version across tests */
+  private IPVersion ipVersion() {
+    return random().nextBoolean() ? IPVersion.IPv4 : IPVersion.IPv6;
+  }
+
+  @Override
+  public void testRandomTiny() throws Exception {
+    ipVersion = ipVersion();
+    super.testRandomTiny();
+  }
+
+  @Override
+  public void testMultiValued() throws Exception {
+    ipVersion = ipVersion();
+    super.testRandomMedium();
+  }
+
+  @Override
+  public void testRandomMedium() throws Exception {
+    ipVersion = ipVersion();
+    super.testMultiValued();
+  }
+
+  @Nightly
+  @Override
+  public void testRandomBig() throws Exception {
+    ipVersion = ipVersion();
+    super.testRandomBig();
+  }
+
+  /** return random range */
+  @Override
+  protected InetAddressRangeField newRangeField(Range r) {
+    return new InetAddressRangeField(FIELD_NAME, ((IpRange)r).min, ((IpRange)r).max);
+  }
+
+  /** return random intersects query */
+  @Override
+  protected Query newIntersectsQuery(Range r) {
+    return InetAddressRangeField.newIntersectsQuery(FIELD_NAME, ((IpRange)r).min, ((IpRange)r).max);
+  }
+
+  /** return random contains query */
+  @Override
+  protected Query newContainsQuery(Range r) {
+    return InetAddressRangeField.newContainsQuery(FIELD_NAME, ((IpRange)r).min, ((IpRange)r).max);
+  }
+
+  /** return random within query */
+  @Override
+  protected Query newWithinQuery(Range r) {
+    return InetAddressRangeField.newWithinQuery(FIELD_NAME, ((IpRange)r).min, ((IpRange)r).max);
+  }
+
+  /** return random crosses query */
+  @Override
+  protected Query newCrossesQuery(Range r) {
+    return InetAddressRangeField.newCrossesQuery(FIELD_NAME, ((IpRange)r).min, ((IpRange)r).max);
+  }
+
+  /** encapsulated IpRange for test validation */
+  private class IpRange extends Range {
+    InetAddress min;
+    InetAddress max;
+
+    IpRange(InetAddress min, InetAddress max) {
+      this.min = min;
+      this.max = max;
+    }
+
+    @Override
+    protected int numDimensions() {
+      return 1;
+    }
+
+    @Override
+    protected InetAddress getMin(int dim) {
+      return min;
+    }
+
+    @Override
+    protected void setMin(int dim, Object val) {
+      byte[] v = ((InetAddress)val).getAddress();
+
+      if (StringHelper.compare(v.length, min.getAddress(), 0, v, 0) < 0) {
+        max = (InetAddress)val;
+      } else {
+        min = (InetAddress) val;
+      }
+    }
+
+    @Override
+    protected InetAddress getMax(int dim) {
+      return max;
+    }
+
+    @Override
+    protected void setMax(int dim, Object val) {
+      byte[] v = ((InetAddress)val).getAddress();
+
+      if (StringHelper.compare(v.length, max.getAddress(), 0, v, 0) > 0) {
+        min = (InetAddress)val;
+      } else {
+        max = (InetAddress) val;
+      }
+    }
+
+    @Override
+    protected boolean isEqual(Range o) {
+      IpRange other = (IpRange)o;
+      return this.min.equals(other.min) && this.max.equals(other.max);
+    }
+
+    @Override
+    protected boolean isDisjoint(Range o) {
+      IpRange other = (IpRange)o;
+      byte[] bMin = min.getAddress();
+      byte[] bMax = max.getAddress();
+      return StringHelper.compare(bMin.length, bMin, 0, other.max.getAddress(), 0) > 0 ||
+          StringHelper.compare(bMax.length, bMax, 0, other.min.getAddress(), 0) < 0;
+    }
+
+    @Override
+    protected boolean isWithin(Range o) {
+      IpRange other = (IpRange)o;
+      byte[] bMin = min.getAddress();
+      byte[] bMax = max.getAddress();
+      return StringHelper.compare(bMin.length, bMin, 0, other.min.getAddress(), 0) >= 0 &&
+          StringHelper.compare(bMax.length, bMax, 0, other.max.getAddress(), 0) <= 0;
+    }
+
+    @Override
+    protected boolean contains(Range o) {
+      IpRange other = (IpRange)o;
+      byte[] bMin = min.getAddress();
+      byte[] bMax = max.getAddress();
+      return StringHelper.compare(bMin.length, bMin, 0, other.min.getAddress(), 0) <= 0 &&
+          StringHelper.compare(bMax.length, bMax, 0, other.max.getAddress(), 0) >= 0;
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder b = new StringBuilder();
+      b.append("Box(");
+      b.append(min.getHostAddress());
+      b.append(" TO ");
+      b.append(max.getHostAddress());
+      b.append(")");
+      return b.toString();
+    }
+  }
+}


[14/43] lucene-solr:feature/autoscaling: SOLR-10235: fix precommit

Posted by sh...@apache.org.
SOLR-10235: fix precommit


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

Branch: refs/heads/feature/autoscaling
Commit: d8442070cfbb0cec815a465a6c143e2b31870e34
Parents: 4d7bc94
Author: Uwe Schindler <us...@apache.org>
Authored: Tue Mar 7 22:07:13 2017 +0100
Committer: Uwe Schindler <us...@apache.org>
Committed: Tue Mar 7 22:07:13 2017 +0100

----------------------------------------------------------------------
 .../org/apache/solr/handler/dataimport/TestJdbcDataSource.java     | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8442070/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestJdbcDataSource.java
----------------------------------------------------------------------
diff --git a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestJdbcDataSource.java b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestJdbcDataSource.java
index dcb4dbc..e9908f9 100644
--- a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestJdbcDataSource.java
+++ b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestJdbcDataSource.java
@@ -35,6 +35,7 @@ import java.util.Properties;
 
 import javax.sql.DataSource;
 
+import org.apache.solr.common.util.SuppressForbidden;
 import org.apache.solr.handler.dataimport.JdbcDataSource.ResultSetIterator;
 import static org.mockito.Mockito.*;
 import org.junit.After;
@@ -635,6 +636,7 @@ public class TestJdbcDataSource extends AbstractDataImportHandlerTestCase {
       return 0;
     }
     
+    @SuppressForbidden(reason="Required by JDBC")
     @Override
     public java.util.logging.Logger getParentLogger() throws java.sql.SQLFeatureNotSupportedException {
       throw new java.sql.SQLFeatureNotSupportedException();


[03/43] lucene-solr:feature/autoscaling: SOLR-9986: Implement DatePointField

Posted by sh...@apache.org.
SOLR-9986: Implement DatePointField


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

Branch: refs/heads/feature/autoscaling
Commit: 3131ec2d99401c1fd1fc33a00343a59a78ab6445
Parents: 6df17c8
Author: Cao Manh Dat <da...@apache.org>
Authored: Tue Mar 7 10:11:47 2017 +0700
Committer: Cao Manh Dat <da...@apache.org>
Committed: Tue Mar 7 10:11:47 2017 +0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   2 +
 .../handler/component/RangeFacetRequest.java    |   5 +-
 .../component/SortedDateStatsValues.java        |  89 +++
 .../handler/component/StatsValuesFactory.java   |   8 +-
 .../org/apache/solr/request/NumericFacets.java  |   5 +-
 .../org/apache/solr/response/DocsStreamer.java  |   2 +
 .../org/apache/solr/schema/DatePointField.java  | 219 ++++++++
 .../apache/solr/search/SolrIndexSearcher.java   |   2 +-
 .../conf/schema-docValuesFaceting.xml           |   5 +-
 .../solr/collection1/conf/schema-point.xml      |  11 +
 .../solr/collection1/conf/schema-sorts.xml      |  11 +-
 .../test-files/solr/collection1/conf/schema.xml |  11 +-
 .../solr/collection1/conf/schema11.xml          |   3 +-
 .../solr/collection1/conf/schema12.xml          |   9 +-
 .../solr/collection1/conf/schema_latest.xml     |  13 +-
 ...lrconfig-parsing-update-processor-chains.xml |   7 +
 .../conf/solrconfig-update-processor-chains.xml |   2 +
 .../handler/admin/LukeRequestHandlerTest.java   |   4 +-
 .../org/apache/solr/schema/DateFieldTest.java   |  10 +-
 .../org/apache/solr/schema/TestPointFields.java | 561 ++++++++++++++++++-
 .../apache/solr/search/TestSolrQueryParser.java |  12 +-
 .../update/processor/AtomicUpdatesTest.java     |  25 +-
 .../ParsingFieldUpdateProcessorsTest.java       |  11 +-
 .../java/org/apache/solr/SolrTestCaseJ4.java    |   2 +
 24 files changed, 966 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 4cfcb72..db721da 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -154,6 +154,8 @@ New Features
 * SOLR-9999: Instrument DirectUpdateHandler2. This registers existing statistics under metrics API and adds
   more metrics to track the rates of update and delete commands. (ab)
 
+* SOLR-9986: Implement DatePointField (Cao Manh Dat, Tom�s Fern�ndez L�bbe)
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/core/src/java/org/apache/solr/handler/component/RangeFacetRequest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/RangeFacetRequest.java b/solr/core/src/java/org/apache/solr/handler/component/RangeFacetRequest.java
index aa3e3cb..3ac7300 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/RangeFacetRequest.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/RangeFacetRequest.java
@@ -31,6 +31,7 @@ import org.apache.solr.common.params.RequiredSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.schema.DatePointField;
 import org.apache.solr.schema.DateRangeField;
 import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.IndexSchema;
@@ -725,7 +726,9 @@ public class RangeFacetRequest extends FacetComponent.FacetBase {
                                        final Date now) {
       super(rangeFacetRequest);
       this.now = now;
-      if (!(field.getType() instanceof TrieDateField) && !(field.getType() instanceof DateRangeField)) {
+      if (!(field.getType() instanceof TrieDateField)
+          && !(field.getType() instanceof DateRangeField)
+          && !(field.getType() instanceof DatePointField)) {
         throw new IllegalArgumentException(TYPE_ERR_MSG);
       }
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/core/src/java/org/apache/solr/handler/component/SortedDateStatsValues.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/SortedDateStatsValues.java b/solr/core/src/java/org/apache/solr/handler/component/SortedDateStatsValues.java
new file mode 100644
index 0000000..0df45c7
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/component/SortedDateStatsValues.java
@@ -0,0 +1,89 @@
+/*
+ * 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.component;
+
+import java.io.IOException;
+import java.util.Date;
+import java.util.Map;
+
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.SortedNumericDocValues;
+import org.apache.lucene.util.BytesRef;
+import org.apache.solr.common.util.NamedList;
+
+public class SortedDateStatsValues implements StatsValues {
+
+  private final DateStatsValues dsv;
+  private final String fieldName;
+  private SortedNumericDocValues sndv;
+
+
+  public SortedDateStatsValues(DateStatsValues dsv, StatsField field) {
+    this.dsv = dsv;
+    this.fieldName = field.getSchemaField().getName();
+  }
+
+  @Override
+  public void accumulate(NamedList stv) {
+    dsv.accumulate(stv);
+  }
+
+  @Override
+  public void accumulate(int docId) throws IOException {
+    if (!sndv.advanceExact(docId)) {
+      missing();
+    } else {
+      for (int i = 0 ; i < sndv.docValueCount(); i++) {
+        dsv.accumulate(new Date(sndv.nextValue()), 1);
+      }
+    }
+
+  }
+
+  @Override
+  public void accumulate(BytesRef value, int count) {
+    dsv.accumulate(value, count);
+  }
+
+  @Override
+  public void missing() {
+    dsv.missing();
+  }
+
+  @Override
+  public void addMissing(int count) {
+    dsv.addMissing(count);
+  }
+
+  @Override
+  public void addFacet(String facetName, Map<String,StatsValues> facetValues) {
+    dsv.addFacet(facetName, facetValues);
+  }
+
+  @Override
+  public NamedList<?> getStatsValues() {
+    return dsv.getStatsValues();
+  }
+
+  @Override
+  public void setNextReader(LeafReaderContext ctx) throws IOException {
+    sndv = DocValues.getSortedNumeric(ctx.reader(), fieldName);
+    assert sndv != null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java b/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java
index 2a6e795..d39ada2 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java
@@ -63,8 +63,12 @@ public class StatsValuesFactory {
     
     final FieldType fieldType = sf.getType(); // TODO: allow FieldType to provide impl.
     
-    if (TrieDateField.class.isInstance(fieldType)) {
-      return new DateStatsValues(statsField);
+    if (TrieDateField.class.isInstance(fieldType) || DatePointField.class.isInstance(fieldType)) {
+      DateStatsValues statsValues = new DateStatsValues(statsField);
+      if (sf.multiValued()) {
+        return new SortedDateStatsValues(statsValues, statsField);
+      }
+      return statsValues;
     } else if (TrieField.class.isInstance(fieldType) || PointField.class.isInstance(fieldType)) {
       
       NumericStatsValues statsValue = new NumericStatsValues(statsField);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/core/src/java/org/apache/solr/request/NumericFacets.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/request/NumericFacets.java b/solr/core/src/java/org/apache/solr/request/NumericFacets.java
index a72eeee..c3bcb9f 100644
--- a/solr/core/src/java/org/apache/solr/request/NumericFacets.java
+++ b/solr/core/src/java/org/apache/solr/request/NumericFacets.java
@@ -19,6 +19,7 @@ package org.apache.solr.request;
 import java.io.IOException;
 import java.util.ArrayDeque;
 import java.util.Collections;
+import java.util.Date;
 import java.util.Deque;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -51,6 +52,7 @@ import org.apache.solr.schema.TrieField;
 import org.apache.solr.search.DocIterator;
 import org.apache.solr.search.DocSet;
 import org.apache.solr.search.SolrIndexSearcher;
+import org.apache.solr.util.DateMathParser;
 
 /** Utility class to compute facets on numeric fields. */
 final class NumericFacets {
@@ -512,7 +514,8 @@ final class NumericFacets {
         return String.valueOf(NumericUtils.sortableIntToFloat((int)bits));
       case DOUBLE:
         return String.valueOf(NumericUtils.sortableLongToDouble(bits));
-        //TODO: DATE
+      case DATE:
+        return new Date(bits).toInstant().toString();
       default:
         throw new AssertionError("Unsupported NumberType: " + fieldType.getNumberType());
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/core/src/java/org/apache/solr/response/DocsStreamer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/response/DocsStreamer.java b/solr/core/src/java/org/apache/solr/response/DocsStreamer.java
index ef0b0c7..bdea9ec 100644
--- a/solr/core/src/java/org/apache/solr/response/DocsStreamer.java
+++ b/solr/core/src/java/org/apache/solr/response/DocsStreamer.java
@@ -31,6 +31,7 @@ import org.apache.solr.common.SolrException;
 import org.apache.solr.response.transform.DocTransformer;
 import org.apache.solr.schema.BinaryField;
 import org.apache.solr.schema.BoolField;
+import org.apache.solr.schema.DatePointField;
 import org.apache.solr.schema.DoublePointField;
 import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.FloatPointField;
@@ -221,6 +222,7 @@ public class DocsStreamer implements Iterator<SolrDocument> {
     KNOWN_TYPES.add(LongPointField.class);
     KNOWN_TYPES.add(DoublePointField.class);
     KNOWN_TYPES.add(FloatPointField.class);
+    KNOWN_TYPES.add(DatePointField.class);
     // We do not add UUIDField because UUID object is not a supported type in JavaBinCodec
     // and if we write UUIDField.toObject, we wouldn't know how to handle it in the client side
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/core/src/java/org/apache/solr/schema/DatePointField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/DatePointField.java b/solr/core/src/java/org/apache/solr/schema/DatePointField.java
new file mode 100644
index 0000000..18bf651
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/schema/DatePointField.java
@@ -0,0 +1,219 @@
+/*
+ * 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.schema;
+
+import java.lang.invoke.MethodHandles;
+import java.time.Instant;
+import java.util.Collection;
+import java.util.Date;
+
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.document.StoredField;
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.legacy.LegacyNumericRangeQuery;
+import org.apache.lucene.legacy.LegacyNumericType;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.queries.function.valuesource.LongFieldSource;
+import org.apache.lucene.queries.function.valuesource.MultiValuedLongFieldSource;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.search.SortedNumericSelector;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.mutable.MutableValueDate;
+import org.apache.lucene.util.mutable.MutableValueLong;
+import org.apache.solr.search.QParser;
+import org.apache.solr.uninverting.UninvertingReader;
+import org.apache.solr.util.DateMathParser;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DatePointField extends PointField implements DateValueFieldType {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public DatePointField() {
+    type = NumberType.DATE;
+  }
+
+
+  @Override
+  public Object toNativeType(Object val) {
+    if (val instanceof String) {
+      return DateMathParser.parseMath(null, (String) val);
+    }
+    return super.toNativeType(val);
+  }
+
+  @Override
+  public Query getPointRangeQuery(QParser parser, SchemaField field, String min, String max, boolean minInclusive, boolean maxInclusive) {
+    long actualMin, actualMax;
+    if (min == null) {
+      actualMin = Long.MIN_VALUE;
+    } else {
+      actualMin = DateMathParser.parseMath(null, min).getTime();
+      if (!minInclusive) {
+        actualMin++;
+      }
+    }
+    if (max == null) {
+      actualMax = Long.MAX_VALUE;
+    } else {
+      actualMax = DateMathParser.parseMath(null, max).getTime();
+      if (!maxInclusive) {
+        actualMax--;
+      }
+    }
+    return LongPoint.newRangeQuery(field.getName(), actualMin, actualMax);
+  }
+
+  @Override
+  public Object toObject(SchemaField sf, BytesRef term) {
+    return new Date(LongPoint.decodeDimension(term.bytes, term.offset));
+  }
+
+  @Override
+  public Object toObject(IndexableField f) {
+    final Number val = f.numericValue();
+    if (val != null) {
+      return new Date(val.longValue());
+    } else {
+      throw new AssertionError("Unexpected state. Field: '" + f + "'");
+    }
+  }
+
+  @Override
+  protected Query getExactQuery(SchemaField field, String externalVal) {
+    return LongPoint.newExactQuery(field.getName(), DateMathParser.parseMath(null, externalVal).getTime());
+  }
+
+  @Override
+  public Query getSetQuery(QParser parser, SchemaField field, Collection<String> externalVals) {
+    assert externalVals.size() > 0;
+    long[] values = new long[externalVals.size()];
+    int i = 0;
+    for (String val:externalVals) {
+      values[i] = DateMathParser.parseMath(null, val).getTime();
+      i++;
+    }
+    return LongPoint.newSetQuery(field.getName(), values);
+  }
+
+  @Override
+  protected String indexedToReadable(BytesRef indexedForm) {
+    return Instant.ofEpochMilli(LongPoint.decodeDimension(indexedForm.bytes, indexedForm.offset)).toString();
+  }
+
+  @Override
+  public void readableToIndexed(CharSequence val, BytesRefBuilder result) {
+    Date date = (Date) toNativeType(val.toString());
+    result.grow(Long.BYTES);
+    result.setLength(Long.BYTES);
+    LongPoint.encodeDimension(date.getTime(), result.bytes(), 0);
+  }
+
+  @Override
+  public SortField getSortField(SchemaField field, boolean top) {
+    field.checkSortability();
+
+    Object missingValue = null;
+    boolean sortMissingLast = field.sortMissingLast();
+    boolean sortMissingFirst = field.sortMissingFirst();
+
+    if (sortMissingLast) {
+      missingValue = top ? Long.MIN_VALUE : Long.MAX_VALUE;
+    } else if (sortMissingFirst) {
+      missingValue = top ? Long.MAX_VALUE : Long.MIN_VALUE;
+    }
+    SortField sf = new SortField(field.getName(), SortField.Type.LONG, top);
+    sf.setMissingValue(missingValue);
+    return sf;
+  }
+
+  @Override
+  public UninvertingReader.Type getUninversionType(SchemaField sf) {
+    if (sf.multiValued()) {
+      return UninvertingReader.Type.SORTED_LONG;
+    } else {
+      return UninvertingReader.Type.LONG_POINT;
+    }
+  }
+
+  @Override
+  public ValueSource getValueSource(SchemaField field, QParser parser) {
+    field.checkFieldCacheSource();
+    return new DatePointFieldSource(field.getName());
+  }
+
+  @Override
+  protected ValueSource getSingleValueSource(SortedNumericSelector.Type choice, SchemaField field) {
+    return new MultiValuedLongFieldSource(field.getName(), choice);
+  }
+
+  @Override
+  public LegacyNumericType getNumericType() {
+    return LegacyNumericType.LONG;
+  }
+
+  @Override
+  public IndexableField createField(SchemaField field, Object value) {
+    if (!isFieldUsed(field)) return null;
+
+    Date date = (value instanceof Date)
+        ? ((Date)value)
+        : DateMathParser.parseMath(null, value.toString());
+    return new LongPoint(field.getName(), date.getTime());
+  }
+
+  @Override
+  protected StoredField getStoredField(SchemaField sf, Object value) {
+    return new StoredField(sf.getName(), ((Date) this.toNativeType(value)).getTime());
+  }
+}
+
+class DatePointFieldSource extends LongFieldSource {
+
+  public DatePointFieldSource(String field) {
+    super(field);
+  }
+
+  @Override
+  public String description() {
+    return "date(" + field + ')';
+  }
+
+  @Override
+  protected MutableValueLong newMutableValueLong() {
+    return new MutableValueDate();
+  }
+
+  @Override
+  public Date longToObject(long val) {
+    return new Date(val);
+  }
+
+  @Override
+  public String longToString(long val) {
+    return longToObject(val).toInstant().toString();
+  }
+
+  @Override
+  public long externalToLong(String extVal) {
+    return DateMathParser.parseMath(null, extVal).getTime();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
index c650845..521324a 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
@@ -923,7 +923,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
                     outValues.add(NumericUtils.sortableLongToDouble(number));
                     break;
                   case DATE:
-                    newVal = new Date(number);
+                    outValues.add(new Date(number));
                     break;
                   default:
                     throw new AssertionError("Unexpected PointType: " + type);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/core/src/test-files/solr/collection1/conf/schema-docValuesFaceting.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-docValuesFaceting.xml b/solr/core/src/test-files/solr/collection1/conf/schema-docValuesFaceting.xml
index 597f2c3..0917ff5 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema-docValuesFaceting.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema-docValuesFaceting.xml
@@ -27,6 +27,7 @@
   <fieldType name="plong" class="solr.LongPointField"/>
   <fieldType name="pdouble" class="solr.DoublePointField"/>
   <fieldType name="pfloat" class="solr.FloatPointField"/>
+  <fieldType name="pdate" class="solr.DatePointField"/>
 
   <field name="id" type="string" indexed="true" stored="true" docValues="false" multiValued="false" required="true"/>
   <field name="id_dv" type="string" indexed="false" stored="false" docValues="true" multiValued="false"
@@ -60,8 +61,8 @@
   <dynamicField name="*_ds_dv" type="double" indexed="true" stored="false" docValues="true" multiValued="true"/>
   <dynamicField name="*_ds_p" type="pdouble" indexed="true" stored="false" docValues="true" multiValued="true"/>
   <dynamicField name="*_dt" type="date" indexed="true" stored="false" docValues="false"/>
-  <dynamicField name="*_dt_dv" type="date" indexed="true" stored="false" docValues="true"/>
-  <dynamicField name="*_dts_dv" type="date" indexed="true" stored="false" docValues="true" multiValued="true"/>
+  <dynamicField name="*_dt_dv" type="${solr.tests.dateClass:pdate}" indexed="true" stored="false" docValues="true"/>
+  <dynamicField name="*_dts_dv" type="${solr.tests.dateClass:pdate}" indexed="true" stored="false" docValues="true" multiValued="true"/>
 
   <defaultSearchField>id</defaultSearchField>
   <uniqueKey>id</uniqueKey>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/core/src/test-files/solr/collection1/conf/schema-point.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-point.xml b/solr/core/src/test-files/solr/collection1/conf/schema-point.xml
index 3561013..ed169a1 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema-point.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema-point.xml
@@ -24,6 +24,7 @@
     <fieldType name="plong" class="solr.LongPointField"/>
     <fieldType name="pdouble" class="solr.DoublePointField"/>
     <fieldType name="pfloat" class="solr.FloatPointField"/>
+    <fieldType name="pdate" class="solr.DatePointField"/>
     
     <fieldType name="int" class="solr.TrieIntField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
     <fieldType name="float" class="solr.TrieFloatField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
@@ -73,21 +74,31 @@
    <dynamicField name="*_p_f_mv_dv"  type="pfloat"    indexed="true"  stored="true" docValues="true" multiValued="true"/>
    <dynamicField name="*_p_f_ni_dv"  type="pfloat"    indexed="false"  stored="true" docValues="true"/>
    <dynamicField name="*_p_f_ni_mv_dv"  type="pfloat"    indexed="false"  stored="true" docValues="true" multiValued="true"/>
+
+   <dynamicField name="*_p_dt"  type="pdate"    indexed="true"  stored="true"/>
+   <dynamicField name="*_p_dt_dv"  type="pdate"    indexed="true"  stored="true" docValues="true"/>
+   <dynamicField name="*_p_dt_mv"  type="pdate"    indexed="true"  stored="true" multiValued="true"/>
+   <dynamicField name="*_p_dt_mv_dv"  type="pdate"    indexed="true"  stored="true" docValues="true" multiValued="true"/>
+   <dynamicField name="*_p_dt_ni_dv"  type="pdate"    indexed="false"  stored="true" docValues="true"/>
+   <dynamicField name="*_p_dt_ni_mv_dv"  type="pdate"    indexed="false"  stored="true" docValues="true" multiValued="true"/>
    
    <!-- return DV fields as  -->
    <dynamicField name="*_p_i_dv_ns"  type="pint"    indexed="true"  stored="false" docValues="true" useDocValuesAsStored="true"/>
    <dynamicField name="*_p_l_dv_ns"  type="plong"    indexed="true"  stored="false" docValues="true" useDocValuesAsStored="true"/>
    <dynamicField name="*_p_d_dv_ns"  type="pdouble"    indexed="true"  stored="false" docValues="true" useDocValuesAsStored="true"/>
    <dynamicField name="*_p_f_dv_ns"  type="pfloat"    indexed="true"  stored="false" docValues="true" useDocValuesAsStored="true"/>
+   <dynamicField name="*_p_dt_dv_ns"  type="pdate"    indexed="true"  stored="false" docValues="true" useDocValuesAsStored="true"/>
    <dynamicField name="*_p_i_ni_ns_dv" type="pint"    indexed="false"  stored="false" docValues="true" useDocValuesAsStored="true"/>
    <dynamicField name="*_p_l_ni_ns_dv" type="plong"   indexed="false"  stored="false" docValues="true" useDocValuesAsStored="true"/>
    <dynamicField name="*_p_d_ni_ns_dv" type="pdouble" indexed="false"  stored="false" docValues="true" useDocValuesAsStored="true"/>
    <dynamicField name="*_p_f_ni_ns_dv" type="pfloat"  indexed="false"  stored="false" docValues="true" useDocValuesAsStored="true"/>
+   <dynamicField name="*_p_dt_ni_ns_dv" type="pdate"   indexed="false"  stored="false" docValues="true" useDocValuesAsStored="true"/>
 
    <dynamicField name="*_p_i_dv_ns_mv"  type="pint"    indexed="true"  stored="false" docValues="true" useDocValuesAsStored="true" multiValued="true"/>
    <dynamicField name="*_p_d_dv_ns_mv"  type="pdouble"    indexed="true"  stored="false" docValues="true" useDocValuesAsStored="true" multiValued="true"/>
    <dynamicField name="*_p_l_dv_ns_mv"  type="plong"    indexed="true"  stored="false" docValues="true" useDocValuesAsStored="true" multiValued="true"/>
    <dynamicField name="*_p_f_dv_ns_mv"  type="pfloat"    indexed="true"  stored="false" docValues="true" useDocValuesAsStored="true" multiValued="true"/>
+   <dynamicField name="*_p_dt_dv_ns_mv"  type="pdate"    indexed="true"  stored="false" docValues="true" useDocValuesAsStored="true" multiValued="true"/>
 
 
  </fields>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/core/src/test-files/solr/collection1/conf/schema-sorts.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-sorts.xml b/solr/core/src/test-files/solr/collection1/conf/schema-sorts.xml
index f68841c..8497318 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema-sorts.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema-sorts.xml
@@ -73,9 +73,9 @@ NOTE: Tests expect every field in this schema to be sortable.
   <field name="date" type="date"/>
   <field name="date_last" type="date_last"/>
   <field name="date_first" type="date_first"/>
-  <field name="date_dv" type="date_dv"/>
-  <field name="date_dv_last" type="date_dv_last"/>
-  <field name="date_dv_first" type="date_dv_first"/>
+  <field name="date_dv" type="${solr.tests.dateClass:pdate}_dv"/>
+  <field name="date_dv_last" type="${solr.tests.dateClass:pdate}_dv_last"/>
+  <field name="date_dv_first" type="${solr.tests.dateClass:pdate}_dv_first"/>
 
   <field name="uuid" type="uuid"/>
   <field name="uuid_last" type="uuid_last"/>
@@ -276,6 +276,11 @@ NOTE: Tests expect every field in this schema to be sortable.
              sortMissingLast="true"/>
   <fieldType name="date_dv_first" class="solr.TrieDateField" stored="true" indexed="false" docValues="true"
              sortMissingFirst="true"/>
+  <fieldType name="pdate_dv" class="solr.DatePointField" stored="true" indexed="false" docValues="true"/>
+  <fieldType name="pdate_dv_last" class="solr.DatePointField" stored="true" indexed="false" docValues="true"
+             sortMissingLast="true"/>
+  <fieldType name="pdate_dv_first" class="solr.DatePointField" stored="true" indexed="false" docValues="true"
+             sortMissingFirst="true"/>
 
   <fieldType name="uuid" class="solr.UUIDField" stored="true" indexed="true"/>
   <fieldType name="uuid_last" class="solr.UUIDField" stored="true" indexed="true" sortMissingLast="true"/>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/core/src/test-files/solr/collection1/conf/schema.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema.xml b/solr/core/src/test-files/solr/collection1/conf/schema.xml
index c53be9b..8c549a3 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema.xml
@@ -49,6 +49,7 @@
   <fieldType name="plong" class="solr.LongPointField" docValues="true"/>
   <fieldType name="pdouble" class="solr.DoublePointField" docValues="true"/>
   <fieldType name="pfloat" class="solr.FloatPointField" docValues="true"/>
+  <fieldType name="pdate" class="solr.DatePointField" docValues="true"/>
 
   <!-- Field type demonstrating an Analyzer failure -->
   <fieldType name="failtype1" class="solr.TextField">
@@ -569,13 +570,13 @@
 
   <field name="textgap" type="textgap" indexed="true" stored="true"/>
 
-  <field name="timestamp" type="date" indexed="true" stored="true" docValues="true" default="NOW" multiValued="false"/>
+  <field name="timestamp" type="${solr.tests.dateClass:pdate}" indexed="true" stored="true" docValues="true" default="NOW" multiValued="false"/>
   <field name="multiDefault" type="string" indexed="true" stored="true" default="muLti-Default" multiValued="true"/>
   <field name="intDefault" type="${solr.tests.intClass:pint}" indexed="true" stored="true" default="42" multiValued="false"/>
   <field name="intDvoDefault" type="${solr.tests.intClass:pint}" indexed="false" stored="false" multiValued="false"
          useDocValuesAsStored="true" docValues="true" default="42" />
   <field name="intRemove" type="${solr.tests.intClass:pint}" indexed="true" stored="true" multiValued="true"/>
-  <field name="dateRemove" type="date" indexed="true" stored="true" multiValued="true"/>
+  <field name="dateRemove" type="${solr.tests.dateClass:pdate}" indexed="true" stored="true" multiValued="true"/>
   <field name="floatRemove" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="true" multiValued="true"/>
 
   <field name="nopositionstext" type="nopositions" indexed="true" stored="true"/>
@@ -621,8 +622,8 @@
   <dynamicField name="*_d" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="true"/>
   <dynamicField name="*_d1" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="true" multiValued="false"/>
   <dynamicField name="*_d1_ndv" type="${solr.tests.doubleClass:pdouble}" indexed="true" docValues="false" stored="true" multiValued="false"/>
-  <dynamicField name="*_dt" type="date" indexed="true" stored="true"/>
-  <dynamicField name="*_dt1" type="date" indexed="true" stored="true" multiValued="false"/>
+  <dynamicField name="*_dt" type="${solr.tests.dateClass:pdate}" indexed="true" stored="true"/>
+  <dynamicField name="*_dt1" type="${solr.tests.dateClass:pdate}" indexed="true" stored="true" multiValued="false"/>
 
 
   <!-- some trie-coded dynamic fields for faster range queries -->
@@ -683,7 +684,7 @@
   <dynamicField name="*_l_dv" type="${solr.tests.longClass:plong}" indexed="true" stored="true" docValues="true"/>
   <dynamicField name="*_f_dv" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="true" docValues="true"/>
   <dynamicField name="*_d_dv" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="true" docValues="true"/>
-  <dynamicField name="*_dt_dv" type="date" indexed="true" stored="true" docValues="true"/>
+  <dynamicField name="*_dt_dv" type="${solr.tests.dateClass:pdate}" indexed="true" stored="true" docValues="true"/>
   <dynamicField name="*_f1_dv" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="true" docValues="true" multiValued="false"/>
 
   <!--  Non-stored, DocValues=true -->

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/core/src/test-files/solr/collection1/conf/schema11.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema11.xml b/solr/core/src/test-files/solr/collection1/conf/schema11.xml
index 24129ae..7591c96 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema11.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema11.xml
@@ -83,6 +83,7 @@
   <fieldType name="plong" class="solr.LongPointField" docValues="true"/>
   <fieldType name="pdouble" class="solr.DoublePointField" docValues="true"/>
   <fieldType name="pfloat" class="solr.FloatPointField" docValues="true"/>
+  <fieldType name="pdate" class="solr.DatePointField" docValues="true"/>
 
     <!-- The format for this date field is of the form 1995-12-31T23:59:59Z, and
          is a more restricted form of the canonical representation of dateTime
@@ -403,7 +404,7 @@ valued. -->
 
    <dynamicField name="*_t"  type="text"    indexed="true"  stored="true"/>
    <dynamicField name="*_b"  type="boolean" indexed="true"  stored="true"/>
-   <dynamicField name="*_dt" type="date"    indexed="true"  stored="true"/>
+   <dynamicField name="*_dt" type="${solr.tests.dateClass:pdate}"    indexed="true"  stored="true"/>
    <dynamicField name="*_ws" type="text_ws" indexed="true"  stored="true"/>
    
    <!-- for testing tfidf functions, see TestFunctionQuery.testTFIDFFunctions -->

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/core/src/test-files/solr/collection1/conf/schema12.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema12.xml b/solr/core/src/test-files/solr/collection1/conf/schema12.xml
index 2d0615c..8577440 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema12.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema12.xml
@@ -42,12 +42,13 @@
   <fieldType name="tfloat" class="solr.TrieFloatField" precisionStep="8" omitNorms="true" positionIncrementGap="0"/>
   <fieldType name="tlong" class="solr.TrieLongField" precisionStep="8" omitNorms="true" positionIncrementGap="0"/>
   <fieldType name="tdouble" class="solr.TrieDoubleField" precisionStep="8" omitNorms="true" positionIncrementGap="0"/>
-  
+
   <!-- Point Fields -->
   <fieldType name="pint" class="solr.IntPointField" docValues="true"/>
   <fieldType name="plong" class="solr.LongPointField" docValues="true"/>
   <fieldType name="pdouble" class="solr.DoublePointField" docValues="true"/>
   <fieldType name="pfloat" class="solr.FloatPointField" docValues="true"/>
+  <fieldType name="pdate" class="solr.DatePointField" docValues="true"/>
 
   <!-- Field type demonstrating an Analyzer failure -->
   <fieldType name="failtype1" class="solr.TextField">
@@ -361,7 +362,7 @@
     </analyzer>
   </fieldType>
 
-  <!-- a text field with the stop filter only on the query analyzer 
+  <!-- a text field with the stop filter only on the query analyzer
    -->
   <fieldType name="text_sw" class="solr.TextField" positionIncrementGap="100">
     <analyzer type="index">
@@ -568,8 +569,8 @@
   <dynamicField name="*_b" type="boolean" indexed="true" stored="true"/>
   <dynamicField name="*_f" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="true"/>
   <dynamicField name="*_d" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="true"/>
-  <dynamicField name="*_dt" type="date" indexed="true" stored="true"/>
-  
+  <dynamicField name="*_dt" type="${solr.tests.dateClass:pdate}" indexed="true" stored="true"/>
+
   <dynamicField name="*_pi" type="pint" indexed="true" stored="true" docValues="false" multiValued="false"/>
 
   <!-- some trie-coded dynamic fields for faster range queries -->

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml b/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml
index e4747d8..dfeac32 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml
@@ -221,10 +221,10 @@
   <dynamicField name="*_ds" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="true" multiValued="true"/>
   <dynamicField name="*_dd" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="false" docValues="true"/>
   <dynamicField name="*_dds" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="false" multiValued="true" docValues="true"/>
-  <dynamicField name="*_dt" type="date" indexed="true" stored="true"/>
-  <dynamicField name="*_dts" type="date" indexed="true" stored="true" multiValued="true"/>
-  <dynamicField name="*_dtd" type="date" indexed="true" stored="false" docValues="true"/>
-  <dynamicField name="*_dtds" type="date" indexed="true" stored="false" multiValued="true" docValues="true"/>
+  <dynamicField name="*_dt" type="${solr.tests.dateClass:pdate}" indexed="true" stored="true"/>
+  <dynamicField name="*_dts" type="${solr.tests.dateClass:pdate}" indexed="true" stored="true" multiValued="true"/>
+  <dynamicField name="*_dtd" type="${solr.tests.dateClass:pdate}" indexed="true" stored="false" docValues="true"/>
+  <dynamicField name="*_dtds" type="${solr.tests.dateClass:pdate}" indexed="true" stored="false" multiValued="true" docValues="true"/>
 
   <!-- docvalues and stored (S suffix) -->
   <dynamicField name="*_idS" type="${solr.tests.intClass:pint}" indexed="true" stored="true" docValues="true"/>
@@ -237,8 +237,8 @@
   <dynamicField name="*_fdsS" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="true" multiValued="true" docValues="true"/>
   <dynamicField name="*_ddS" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="true" docValues="true"/>
   <dynamicField name="*_ddsS" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="true" multiValued="true" docValues="true"/>
-  <dynamicField name="*_dtdS" type="date" indexed="true" stored="true" docValues="true"/>
-  <dynamicField name="*_dtdsS" type="date" indexed="true" stored="true" multiValued="true" docValues="true"/>
+  <dynamicField name="*_dtdS" type="${solr.tests.dateClass:pdate}" indexed="true" stored="true" docValues="true"/>
+  <dynamicField name="*_dtdsS" type="${solr.tests.dateClass:pdate}" indexed="true" stored="true" multiValued="true" docValues="true"/>
 
 
   <dynamicField name="*_b" type="boolean" indexed="true" stored="true"/>
@@ -400,6 +400,7 @@
   <fieldType name="plong" class="solr.LongPointField" docValues="true"/>
   <fieldType name="pdouble" class="solr.DoublePointField" docValues="true"/>
   <fieldType name="pfloat" class="solr.FloatPointField" docValues="true"/>
+  <fieldType name="pdate" class="solr.DatePointField" docValues="true"/>
   
 
   <!-- The format for this date field is of the form 1995-12-31T23:59:59Z, and

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/core/src/test-files/solr/collection1/conf/solrconfig-parsing-update-processor-chains.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-parsing-update-processor-chains.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-parsing-update-processor-chains.xml
index 7078da7..f83df6c 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-parsing-update-processor-chains.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-parsing-update-processor-chains.xml
@@ -47,6 +47,13 @@
     </processor>
   </updateRequestProcessorChain>
 
+  <updateRequestProcessorChain name="parse-date-explicit-typeclass-point-selector-no-run-processor">
+    <processor class="solr.ParseDateFieldUpdateProcessorFactory">
+      <str name="typeClass">solr.DatePointField</str>
+      <str name="format">yyyy-MM-dd'T'HH:mm:ss.SSSZ</str>
+    </processor>
+  </updateRequestProcessorChain>
+
   <updateRequestProcessorChain name="parse-date-explicit-typeclass-selector-no-run-processor">
     <processor class="solr.ParseDateFieldUpdateProcessorFactory">
       <str name="typeClass">solr.TrieDateField</str>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/core/src/test-files/solr/collection1/conf/solrconfig-update-processor-chains.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-update-processor-chains.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-update-processor-chains.xml
index a38bc04..426f3c0 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-update-processor-chains.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-update-processor-chains.xml
@@ -94,6 +94,7 @@
       <!-- each set of exclusions is checked independently -->
       <lst name="exclude">
         <str name="typeClass">solr.TrieDateField</str>
+        <str name="typeClass">solr.DatePointField</str>
       </lst>
       <lst name="exclude">
         <str name="fieldRegex">.*HOSS.*</str>
@@ -144,6 +145,7 @@
   <updateRequestProcessorChain name="trim-classes">
     <processor class="solr.TrimFieldUpdateProcessorFactory">
       <str name="typeClass">solr.TrieDateField</str>
+      <str name="typeClass">solr.DatePointField</str>
       <str name="typeClass">solr.StrField</str>
     </processor>
   </updateRequestProcessorChain>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/core/src/test/org/apache/solr/handler/admin/LukeRequestHandlerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/LukeRequestHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/LukeRequestHandlerTest.java
index 92b4943..d253ef2 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/LukeRequestHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/LukeRequestHandlerTest.java
@@ -117,7 +117,7 @@ public class LukeRequestHandlerTest extends AbstractSolrTestCase {
     // only valid for fields that are indexed & stored
     for (String f : Arrays.asList("solr_t","solr_s","solr_ti",
         "solr_td","solr_dt","solr_b")) {
-
+      if (h.getCore().getLatestSchema().getField(f).getType().isPointField()) continue;
       final String xp = getFieldXPathPrefix(f);
       assertQ("Not as many index flags as expected ("+numFlags+") for " + f,
           req("qt","/admin/luke", "fl", f),
@@ -166,7 +166,7 @@ public class LukeRequestHandlerTest extends AbstractSolrTestCase {
       response = h.query(req);
       for (String f : Arrays.asList("solr_t", "solr_s", "solr_ti",
           "solr_td", "solr_dt", "solr_b")) {
-
+        if (h.getCore().getLatestSchema().getField(f).getType().isPointField()) continue;
         assertNull(TestHarness.validateXPath(response,
             getFieldXPathPrefix(f) + "[@name='index']"));
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/core/src/test/org/apache/solr/schema/DateFieldTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/schema/DateFieldTest.java b/solr/core/src/test/org/apache/solr/schema/DateFieldTest.java
index ac451bf..d42d6dd 100644
--- a/solr/core/src/test/org/apache/solr/schema/DateFieldTest.java
+++ b/solr/core/src/test/org/apache/solr/schema/DateFieldTest.java
@@ -29,7 +29,7 @@ import org.apache.solr.core.SolrResourceLoader;
 public class DateFieldTest extends SolrTestCaseJ4 {
   private final String testInstanceDir = TEST_HOME() + File.separator + "collection1";
   private final String testConfHome = testInstanceDir + File.separator + "conf"+ File.separator;
-  private TrieDateField f = null;
+  private FieldType f = null;
 
   @Override
   public void setUp()  throws Exception {
@@ -40,7 +40,7 @@ public class DateFieldTest extends SolrTestCaseJ4 {
     SolrConfig config = new SolrConfig
         (new SolrResourceLoader(Paths.get(testInstanceDir)), testConfHome + "solrconfig.xml", null);
     IndexSchema schema = IndexSchemaFactory.buildIndexSchema(testConfHome + "schema.xml", config);
-    f = new TrieDateField();
+    f = random().nextBoolean()? new TrieDateField() : new DatePointField();
     f.init(schema, Collections.<String,String>emptyMap());
   }
 
@@ -51,13 +51,13 @@ public class DateFieldTest extends SolrTestCaseJ4 {
     SchemaField sf = new SchemaField( "test", f, props, null );
     // String
     IndexableField out = f.createField(sf, "1995-12-31T23:59:59Z" );
-    assertEquals(820454399000L, f.toObject( out ).getTime() );
+    assertEquals(820454399000L, ((Date) f.toObject( out )).getTime() );
     // Date obj
     out = f.createField(sf, new Date(820454399000L) );
-    assertEquals(820454399000L, f.toObject( out ).getTime() );
+    assertEquals(820454399000L, ((Date) f.toObject( out )).getTime() );
     // Date math
     out = f.createField(sf, "1995-12-31T23:59:59.99Z+5MINUTES");
-    assertEquals(820454699990L, f.toObject( out ).getTime() );
+    assertEquals(820454699990L, ((Date) f.toObject( out )).getTime() );
   }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3131ec2d/solr/core/src/test/org/apache/solr/schema/TestPointFields.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/schema/TestPointFields.java b/solr/core/src/test/org/apache/solr/schema/TestPointFields.java
index b3d0b97..3c1f0b3 100644
--- a/solr/core/src/test/org/apache/solr/schema/TestPointFields.java
+++ b/solr/core/src/test/org/apache/solr/schema/TestPointFields.java
@@ -25,6 +25,7 @@ import org.apache.lucene.search.IndexOrDocValuesQuery;
 import org.apache.lucene.search.PointRangeQuery;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.util.DateMathParser;
 import org.junit.After;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -587,6 +588,126 @@ public class TestPointFields extends SolrTestCaseJ4 {
     doTestSetQueries("number_p_l_mv", getRandomStringArrayWithLongs(10, false), true);
     doTestSetQueries("number_p_l_ni_dv", getRandomStringArrayWithLongs(10, false), false);
   }
+
+  // Date
+
+  @Test
+  public void testDatePointFieldExactQuery() throws Exception {
+    doTestDatePointFieldExactQuery("number_p_dt", "1995-12-31T23:59:59Z");
+    doTestDatePointFieldExactQuery("number_p_dt_mv", "2015-12-31T23:59:59Z-1DAY");
+    doTestDatePointFieldExactQuery("number_p_dt_ni_dv", "2000-12-31T23:59:59Z+3DAYS");
+    doTestDatePointFieldExactQuery("number_p_dt_ni_ns_dv", "1995-12-31T23:59:59Z-1MONTH");
+    doTestDatePointFieldExactQuery("number_p_dt_ni_mv_dv", "1995-12-31T23:59:59Z+2MONTHS");
+  }
+
+  @Test
+  public void testDatePointFieldReturn() throws Exception {
+    testPointFieldReturn("number_p_dt", "date",
+        new String[]{"1995-12-31T23:59:59Z", "1994-02-28T23:59:59Z",
+            "2015-12-31T23:59:59Z", "2000-10-31T23:59:59Z", "1999-12-31T12:59:59Z"});
+    clearIndex();
+    assertU(commit());
+    testPointFieldReturn("number_p_dt_dv_ns", "date",
+        new String[]{"1995-12-31T23:59:59Z", "1994-02-28T23:59:59Z",
+            "2015-12-31T23:59:59Z", "2000-10-31T23:59:59Z", "1999-12-31T12:59:59Z"});
+  }
+
+  @Test
+  public void testDatePointFieldRangeQuery() throws Exception {
+    doTestDatePointFieldRangeQuery("number_p_dt");
+    doTestDatePointFieldRangeQuery("number_p_dt_ni_ns_dv");
+  }
+
+  @Test
+  public void testDatePointFieldSort() throws Exception {
+    doTestPointFieldSort("number_p_dt", "number_p_dt_dv", getSequentialStringArrayWithDates(10));
+  }
+
+  @Test
+  public void testDatePointFieldFacetField() throws Exception {
+    testPointFieldFacetField("number_p_dt", "number_p_dt_dv", getSequentialStringArrayWithDates(10));
+    clearIndex();
+    assertU(commit());
+    testPointFieldFacetField("number_p_dt", "number_p_dt_dv", getSequentialStringArrayWithDates(10));
+  }
+
+  @Test
+  public void testDatePointFieldRangeFacet() throws Exception {
+    doTestDatePointFieldRangeFacet("number_p_dt_dv", "number_p_dt");
+  }
+
+  @Test
+  public void testDatePointFunctionQuery() throws Exception {
+    doTestDatePointFunctionQuery("number_p_dt_dv", "number_p_dt", "date");
+  }
+
+  @Test
+  public void testDatePointStats() throws Exception {
+    testDatePointStats("number_p_dt", "number_p_dt_dv", getSequentialStringArrayWithDates(10));
+    testDatePointStats("number_p_dt_mv", "number_p_dt_mv_dv", getSequentialStringArrayWithDates(10));
+  }
+
+  @Test
+  public void testDatePointFieldMultiValuedExactQuery() throws Exception {
+    testPointFieldMultiValuedExactQuery("number_p_dt_mv", getSequentialStringArrayWithDates(20));
+    testPointFieldMultiValuedExactQuery("number_p_dt_ni_mv_dv", getSequentialStringArrayWithDates(20));
+  }
+
+  @Test
+  public void testDatePointFieldMultiValuedReturn() throws Exception {
+    testPointFieldMultiValuedReturn("number_p_dt_mv", "date", getSequentialStringArrayWithDates(20));
+    testPointFieldMultiValuedReturn("number_p_dt_ni_mv_dv", "date", getSequentialStringArrayWithDates(20));
+    testPointFieldMultiValuedReturn("number_p_dt_dv_ns_mv", "date", getSequentialStringArrayWithDates(20));
+  }
+
+  @Test
+  public void testDatePointFieldMultiValuedRangeQuery() throws Exception {
+    testPointFieldMultiValuedRangeQuery("number_p_dt_mv", "date", getSequentialStringArrayWithDates(20));
+    testPointFieldMultiValuedRangeQuery("number_p_dt_ni_mv_dv", "date", getSequentialStringArrayWithDates(20));
+  }
+
+  @Test
+  public void testDatePointFieldMultiValuedFacetField() throws Exception {
+    testPointFieldMultiValuedFacetField("number_p_dt_mv", "number_p_dt_mv_dv", getSequentialStringArrayWithDates(20));
+    testPointFieldMultiValuedFacetField("number_p_dt_mv", "number_p_dt_mv_dv", getRandomStringArrayWithDates(20, false));
+  }
+
+  @Test
+  public void testDatePointFieldMultiValuedRangeFacet() throws Exception {
+    doTestDatePointFieldMultiValuedRangeFacet("number_p_dt_mv_dv", "number_p_dt_mv");
+  }
+
+  @Test
+  public void testDatePointMultiValuedFunctionQuery() throws Exception {
+    testPointMultiValuedFunctionQuery("number_p_dt_mv", "number_p_dt_mv_dv", "date", getSequentialStringArrayWithDates(20));
+  }
+
+  @Test
+  public void testDatePointFieldsAtomicUpdates() throws Exception {
+    if (!Boolean.getBoolean("enable.update.log")) {
+      return;
+    }
+    testDatePointFieldsAtomicUpdates("number_p_dt", "date");
+    testDatePointFieldsAtomicUpdates("number_p_dt_dv", "date");
+    testDatePointFieldsAtomicUpdates("number_p_dt_dv_ns", "date");
+  }
+
+  @Test
+  public void testMultiValuedDatePointFieldsAtomicUpdates() throws Exception {
+    if (!Boolean.getBoolean("enable.update.log")) {
+      return;
+    }
+    testMultiValuedDatePointFieldsAtomicUpdates("number_p_dt_mv", "date");
+    testMultiValuedDatePointFieldsAtomicUpdates("number_p_dt_ni_mv_dv", "date");
+    testMultiValuedDatePointFieldsAtomicUpdates("number_p_dt_dv_ns_mv", "date");
+  }
+
+  @Test
+  public void testDatePointSetQuery() throws Exception {
+    doTestSetQueries("number_p_dt", getRandomStringArrayWithDates(10, false), false);
+    doTestSetQueries("number_p_dt_mv", getRandomStringArrayWithDates(10, false), true);
+    doTestSetQueries("number_p_dt_ni_dv", getRandomStringArrayWithDates(10, false), false);
+  }
   
   @Test
   public void testIndexOrDocValuesQuery() throws Exception {
@@ -664,6 +785,15 @@ public class TestPointFields extends SolrTestCaseJ4 {
     }
     return arr;
   }
+
+  private String[] getSequentialStringArrayWithDates(int length) {
+    assert length < 60;
+    String[] arr = new String[length];
+    for (int i = 0; i < length; i++) {
+      arr[i] = String.format(Locale.ROOT, "1995-12-11T19:59:%02dZ", i);
+    }
+    return arr;
+  }
   
   private String[] getSequentialStringArrayWithDoubles(int length) {
     String[] arr = new String[length];
@@ -718,6 +848,27 @@ public class TestPointFields extends SolrTestCaseJ4 {
     }
     return stringArr;
   }
+
+  private String[] getRandomStringArrayWithDates(int length, boolean sorted) {
+    assert length < 60;
+    Set<Integer> set;
+    if (sorted) {
+      set = new TreeSet<>();
+    } else {
+      set = new HashSet<>();
+    }
+    while (set.size() < length) {
+      int number = random().nextInt(60);
+      set.add(number);
+    }
+    String[] stringArr = new String[length];
+    int i = 0;
+    for (int val:set) {
+      stringArr[i] = String.format(Locale.ROOT, "1995-12-11T19:59:%02dZ", val);
+      i++;
+    }
+    return stringArr;
+  }
   
   private void doTestIntPointFieldExactQuery(String field, boolean testLong) throws Exception {
     for (int i=0; i < 10; i++) {
@@ -1037,12 +1188,21 @@ public class TestPointFields extends SolrTestCaseJ4 {
     }
     assertU(commit());
     for (int i = 0; i < 20; i++) {
-      assertQ(req("q", fieldName + ":" + numbers[i].replace("-", "\\-")), 
-          "//*[@numFound='1']");
+      if (h.getCore().getLatestSchema().getField(fieldName).getType() instanceof DatePointField) {
+        assertQ(req("q", fieldName + ":\"" + numbers[i] + "\""),
+            "//*[@numFound='1']");
+      } else {
+        assertQ(req("q", fieldName + ":" + numbers[i].replace("-", "\\-")),
+            "//*[@numFound='1']");
+      }
     }
     
     for (int i = 0; i < 20; i++) {
-      assertQ(req("q", fieldName + ":" + numbers[i].replace("-", "\\-") + " OR " + fieldName + ":" + numbers[(i+1)%10].replace("-", "\\-")), "//*[@numFound='2']");
+      if (h.getCore().getLatestSchema().getField(fieldName).getType() instanceof DatePointField) {
+        assertQ(req("q", fieldName + ":\"" + numbers[i] + "\"" + " OR " + fieldName + ":\"" + numbers[(i+1)%10]+"\""), "//*[@numFound='2']");
+      } else {
+        assertQ(req("q", fieldName + ":" + numbers[i].replace("-", "\\-") + " OR " + fieldName + ":" + numbers[(i+1)%10].replace("-", "\\-")), "//*[@numFound='2']");
+      }
     }
   }
   
@@ -1089,10 +1249,10 @@ public class TestPointFields extends SolrTestCaseJ4 {
     assertTrue(h.getCore().getLatestSchema().getField(fieldName).multiValued());
     assertTrue(h.getCore().getLatestSchema().getField(fieldName).getType() instanceof PointField);
     for (int i=0; i < 10; i++) {
-      assertU(adoc("id", String.valueOf(i), fieldName, String.valueOf(i), fieldName, String.valueOf(i+10)));
+      assertU(adoc("id", String.valueOf(i), fieldName, numbers[i], fieldName, numbers[i+10]));
     }
     assertU(commit());
-    assertQ(req("q", fieldName + ":[0 TO 3]", "fl", "id, " + fieldName), 
+    assertQ(req("q", String.format(Locale.ROOT, "%s:[%s TO %s]", fieldName, numbers[0], numbers[3]), "fl", "id, " + fieldName),
         "//*[@numFound='4']",
         "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[0] + "']",
         "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[2][.='" + numbers[10] + "']",
@@ -1103,36 +1263,36 @@ public class TestPointFields extends SolrTestCaseJ4 {
         "//result/doc[4]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[3] + "']",
         "//result/doc[4]/arr[@name='" + fieldName + "']/" + type + "[2][.='" + numbers[13] + "']");
     
-    assertQ(req("q", fieldName + ":{0 TO 3]", "fl", "id, " + fieldName), 
+    assertQ(req("q", String.format(Locale.ROOT, "%s:{%s TO %s]", fieldName, numbers[0], numbers[3]), "fl", "id, " + fieldName),
         "//*[@numFound='3']",
         "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[1] + "']",
         "//result/doc[2]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[2] + "']",
         "//result/doc[3]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[3] + "']");
     
-    assertQ(req("q", fieldName + ":[0 TO 3}", "fl", "id, " + fieldName), 
+    assertQ(req("q", String.format(Locale.ROOT, "%s:[%s TO %s}", fieldName, numbers[0], numbers[3]), "fl", "id, " + fieldName),
         "//*[@numFound='3']",
         "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[0] + "']",
         "//result/doc[2]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[1] + "']",
         "//result/doc[3]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[2] + "']");
     
-    assertQ(req("q", fieldName + ":{0 TO 3}", "fl", "id, " + fieldName), 
+    assertQ(req("q", String.format(Locale.ROOT, "%s:{%s TO %s}", fieldName, numbers[0], numbers[3]), "fl", "id, " + fieldName),
         "//*[@numFound='2']",
         "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[1] + "']",
         "//result/doc[2]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[2] + "']");
-    
-    assertQ(req("q", fieldName + ":{0 TO *}", "fl", "id, " + fieldName), 
+
+    assertQ(req("q", String.format(Locale.ROOT, "%s:{%s TO *}", fieldName, numbers[0]), "fl", "id, " + fieldName),
         "//*[@numFound='10']",
         "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[0] + "']");
     
-    assertQ(req("q", fieldName + ":{10 TO *}", "fl", "id, " + fieldName), 
+    assertQ(req("q", String.format(Locale.ROOT, "%s:{%s TO *}", fieldName, numbers[10]), "fl", "id, " + fieldName),
         "//*[@numFound='9']",
         "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[1] + "']");
     
-    assertQ(req("q", fieldName + ":{* TO 3}", "fl", "id, " + fieldName), 
+    assertQ(req("q", String.format(Locale.ROOT, "%s:{* TO %s}", fieldName, numbers[3]), "fl", "id, " + fieldName),
         "//*[@numFound='3']",
         "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[0] + "']");
     
-    assertQ(req("q", fieldName + ":[* TO 3}", "fl", "id, " + fieldName), 
+    assertQ(req("q", String.format(Locale.ROOT, "%s:[* TO %s}", fieldName, numbers[3]), "fl", "id, " + fieldName),
         "//*[@numFound='3']",
         "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[0] + "']");
     
@@ -1141,14 +1301,14 @@ public class TestPointFields extends SolrTestCaseJ4 {
         "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[0] + "']",
         "//result/doc[10]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[9] + "']");
     
-    assertQ(req("q", fieldName + ":[0 TO 1] OR " + fieldName + ":[8 TO 9]", "fl", "id, " + fieldName), 
+    assertQ(req("q", String.format(Locale.ROOT, "%s:[%s TO %s] OR %s:[%s TO %s]", fieldName, numbers[0], numbers[1], fieldName, numbers[8], numbers[9]), "fl", "id, " + fieldName),
         "//*[@numFound='4']",
         "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[0] + "']",
         "//result/doc[2]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[1] + "']",
         "//result/doc[3]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[8] + "']",
         "//result/doc[4]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[9] + "']");
     
-    assertQ(req("q", fieldName + ":[0 TO 0] AND " + fieldName + ":[10 TO 10]", "fl", "id, " + fieldName), 
+    assertQ(req("q", String.format(Locale.ROOT, "%s:[%s TO %s] OR %s:[%s TO %s]", fieldName, numbers[0], numbers[0], fieldName, numbers[10], numbers[10]), "fl", "id, " + fieldName),
         "//*[@numFound='1']",
         "//result/doc[1]/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[0] + "']");
   }
@@ -1238,12 +1398,22 @@ public class TestPointFields extends SolrTestCaseJ4 {
         larger = numbers[1];
       }
     } catch (NumberFormatException e) {
-      if (Double.valueOf(numbers[1]) < Double.valueOf(numbers[2])) {
-        smaller = numbers[1];
-        larger = numbers[2];
-      } else {
-        smaller = numbers[2];
-        larger = numbers[1];
+      try {
+        if (Double.valueOf(numbers[1]) < Double.valueOf(numbers[2])) {
+          smaller = numbers[1];
+          larger = numbers[2];
+        } else {
+          smaller = numbers[2];
+          larger = numbers[1];
+        }
+      } catch (NumberFormatException e2) {
+        if (DateMathParser.parseMath(null, numbers[1]).getTime() < DateMathParser.parseMath(null, numbers[2]).getTime()) {
+          smaller = numbers[1];
+          larger = numbers[2];
+        } else {
+          smaller = numbers[2];
+          larger = numbers[1];
+        }
       }
     }
     
@@ -1818,4 +1988,353 @@ public class TestPointFields extends SolrTestCaseJ4 {
         "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='18'][.='2']",
         "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='-10'][.='0']");
   }
+
+
+  private void doTestDatePointFieldExactQuery(String field, String baseDate) throws Exception {
+    for (int i=0; i < 10; i++) {
+      assertU(adoc("id", String.valueOf(i), field, String.format(Locale.ROOT, "%s+%dMINUTES", baseDate, i+1)));
+    }
+    assertU(commit());
+    for (int i = 0; i < 10; i++) {
+      String date = String.format(Locale.ROOT, "%s+%dMINUTES", baseDate, i+1);
+      assertQ(req("q", field + ":\""+date+"\"", "fl", "id, " + field),
+          "//*[@numFound='1']");
+    }
+
+    for (int i = 0; i < 10; i++) {
+      String date1 = String.format(Locale.ROOT, "%s+%dMINUTES", baseDate, i+1);
+      String date2 = String.format(Locale.ROOT, "%s+%dMINUTES", baseDate, ((i+1)%10 + 1));
+      assertQ(req("q", field + ":\"" + date1 + "\""
+          + " OR " + field + ":\"" + date2 + "\""), "//*[@numFound='2']");
+    }
+
+    clearIndex();
+    assertU(commit());
+  }
+
+  private void doTestDatePointFieldRangeQuery(String fieldName) throws Exception {
+    String baseDate = "1995-12-31T10:59:59Z";
+    for (int i = 0; i < 10; i++) {
+      assertU(adoc("id", String.valueOf(i), fieldName, String.format(Locale.ROOT, "%s+%dHOURS", baseDate, i)));
+    }
+    assertU(commit());
+    assertQ(req("q", fieldName + ":" + String.format(Locale.ROOT, "[%s+0HOURS TO %s+3HOURS]", baseDate, baseDate), "fl", "id, " + fieldName),
+        "//*[@numFound='4']",
+        "//result/doc[1]/date[@name='" + fieldName + "'][.='1995-12-31T10:59:59Z']",
+        "//result/doc[2]/date[@name='" + fieldName + "'][.='1995-12-31T11:59:59Z']",
+        "//result/doc[3]/date[@name='" + fieldName + "'][.='1995-12-31T12:59:59Z']",
+        "//result/doc[4]/date[@name='" + fieldName + "'][.='1995-12-31T13:59:59Z']");
+
+    assertQ(req("q", fieldName + ":" + String.format(Locale.ROOT, "{%s+0HOURS TO %s+3HOURS]", baseDate, baseDate), "fl", "id, " + fieldName),
+        "//*[@numFound='3']",
+        "//result/doc[1]/date[@name='" + fieldName + "'][.='1995-12-31T11:59:59Z']",
+        "//result/doc[2]/date[@name='" + fieldName + "'][.='1995-12-31T12:59:59Z']",
+        "//result/doc[3]/date[@name='" + fieldName + "'][.='1995-12-31T13:59:59Z']");
+
+    assertQ(req("q", fieldName + ":"+ String.format(Locale.ROOT, "[%s+0HOURS TO %s+3HOURS}",baseDate,baseDate), "fl", "id, " + fieldName),
+        "//*[@numFound='3']",
+        "//result/doc[1]/date[@name='" + fieldName + "'][.='1995-12-31T10:59:59Z']",
+        "//result/doc[2]/date[@name='" + fieldName + "'][.='1995-12-31T11:59:59Z']",
+        "//result/doc[3]/date[@name='" + fieldName + "'][.='1995-12-31T12:59:59Z']");
+
+    assertQ(req("q", fieldName + ":"+ String.format(Locale.ROOT, "{%s+0HOURS TO %s+3HOURS}",baseDate,baseDate), "fl", "id, " + fieldName),
+        "//*[@numFound='2']",
+        "//result/doc[1]/date[@name='" + fieldName + "'][.='1995-12-31T11:59:59Z']",
+        "//result/doc[2]/date[@name='" + fieldName + "'][.='1995-12-31T12:59:59Z']");
+
+    assertQ(req("q", fieldName + ":" + String.format(Locale.ROOT, "{%s+0HOURS TO *}",baseDate), "fl", "id, " + fieldName),
+        "//*[@numFound='9']",
+        "//result/doc[1]/date[@name='" + fieldName + "'][.='1995-12-31T11:59:59Z']");
+
+    assertQ(req("q", fieldName + ":" + String.format(Locale.ROOT, "{* TO %s+3HOURS}",baseDate), "fl", "id, " + fieldName),
+        "//*[@numFound='3']",
+        "//result/doc[1]/date[@name='" + fieldName + "'][.='1995-12-31T10:59:59Z']");
+
+    assertQ(req("q", fieldName + ":" + String.format(Locale.ROOT, "[* TO %s+3HOURS}",baseDate), "fl", "id, " + fieldName),
+        "//*[@numFound='3']",
+        "//result/doc[1]/date[@name='" + fieldName + "'][.='1995-12-31T10:59:59Z']");
+
+    assertQ(req("q", fieldName + ":[* TO *}", "fl", "id, " + fieldName),
+        "//*[@numFound='10']",
+        "//result/doc[1]/date[@name='" + fieldName + "'][.='1995-12-31T10:59:59Z']",
+        "//result/doc[10]/date[@name='" + fieldName + "'][.='1995-12-31T19:59:59Z']");
+
+    assertQ(req("q", fieldName + ":" + String.format(Locale.ROOT, "[%s+0HOURS TO %s+1HOURS]",baseDate,baseDate)
+            +" OR " + fieldName + ":" + String.format(Locale.ROOT, "[%s+8HOURS TO %s+9HOURS]",baseDate,baseDate) , "fl", "id, " + fieldName),
+        "//*[@numFound='4']",
+        "//result/doc[1]/date[@name='" + fieldName + "'][.='1995-12-31T10:59:59Z']",
+        "//result/doc[2]/date[@name='" + fieldName + "'][.='1995-12-31T11:59:59Z']",
+        "//result/doc[3]/date[@name='" + fieldName + "'][.='1995-12-31T18:59:59Z']",
+        "//result/doc[4]/date[@name='" + fieldName + "'][.='1995-12-31T19:59:59Z']");
+
+    assertQ(req("q", fieldName + ":"+String.format(Locale.ROOT, "[%s+0HOURS TO %s+1HOURS]",baseDate,baseDate)
+            +" AND " + fieldName + ":"+String.format(Locale.ROOT, "[%s+1HOURS TO %s+2HOURS]",baseDate,baseDate) , "fl", "id, " + fieldName),
+        "//*[@numFound='1']",
+        "//result/doc[1]/date[@name='" + fieldName + "'][.='1995-12-31T11:59:59Z']");
+
+    assertQ(req("q", fieldName + ":"+String.format(Locale.ROOT, "[%s+0HOURS TO %s+1HOURS]",baseDate,baseDate)
+            +" AND NOT " + fieldName + ":"+String.format(Locale.ROOT, "[%s+1HOURS TO %s+2HOURS]",baseDate,baseDate) , "fl", "id, " + fieldName),
+        "//*[@numFound='1']",
+        "//result/doc[1]/date[@name='" + fieldName + "'][.='1995-12-31T10:59:59Z']");
+
+    clearIndex();
+    assertU(commit());
+  }
+
+  private void doTestDatePointFieldRangeFacet(String docValuesField, String nonDocValuesField) throws Exception {
+    String baseDate = "1995-01-10T10:59:59Z";
+    for (int i = 0; i < 10; i++) {
+      String date = String.format(Locale.ROOT, "%s+%dDAYS", baseDate, i);
+      assertU(adoc("id", String.valueOf(i), docValuesField, date, nonDocValuesField, date));
+    }
+    assertU(commit());
+    assertTrue(h.getCore().getLatestSchema().getField(docValuesField).hasDocValues());
+    assertTrue(h.getCore().getLatestSchema().getField(docValuesField).getType() instanceof PointField);
+    assertQ(req("q", "*:*", "facet", "true", "facet.range", docValuesField, "facet.range.start", "1995-01-10T10:59:59Z-10DAYS",
+        "facet.range.end", "1995-01-10T10:59:59Z+10DAYS", "facet.range.gap", "+2DAYS"),
+        "//*[@numFound='10']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-10T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-12T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-14T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-16T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-18T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-08T10:59:59Z'][.='0']");
+
+    assertQ(req("q", "*:*", "facet", "true", "facet.range", docValuesField, "facet.range.start", "1995-01-10T10:59:59Z-10DAYS",
+        "facet.range.end", "1995-01-10T10:59:59Z+10DAYS", "facet.range.gap", "+2DAYS", "facet.range.method", "dv"),
+        "//*[@numFound='10']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-10T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-12T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-14T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-16T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-18T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-08T10:59:59Z'][.='0']");
+
+    assertFalse(h.getCore().getLatestSchema().getField(nonDocValuesField).hasDocValues());
+    assertTrue(h.getCore().getLatestSchema().getField(nonDocValuesField).getType() instanceof PointField);
+    // Range Faceting with method = filter should work
+    assertQ(req("q", "*:*", "facet", "true", "facet.range", nonDocValuesField, "facet.range.start", "1995-01-10T10:59:59Z-10DAYS",
+        "facet.range.end", "1995-01-10T10:59:59Z+10DAYS", "facet.range.gap", "+2DAYS", "facet.range.method", "filter"),
+        "//*[@numFound='10']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-10T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-12T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-14T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-16T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-18T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-08T10:59:59Z'][.='0']");
+
+    // this should actually use filter method instead of dv
+    assertQ(req("q", "*:*", "facet", "true", "facet.range", nonDocValuesField, "facet.range.start", "1995-01-10T10:59:59Z-10DAYS",
+        "facet.range.end", "1995-01-10T10:59:59Z+10DAYS", "facet.range.gap", "+2DAYS", "facet.range.method", "dv"),
+        "//*[@numFound='10']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-10T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-12T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-14T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-16T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-18T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-08T10:59:59Z'][.='0']");
+  }
+
+  private void doTestDatePointFieldMultiValuedRangeFacet(String docValuesField, String nonDocValuesField) throws Exception {
+    String baseDate = "1995-01-10T10:59:59Z";
+    for (int i = 0; i < 10; i++) {
+      String date1 = String.format(Locale.ROOT, "%s+%dDAYS", baseDate, i);
+      String date2 = String.format(Locale.ROOT, "%s+%dDAYS", baseDate, i+10);
+      assertU(adoc("id", String.valueOf(i), docValuesField, date1, docValuesField, date2,
+          nonDocValuesField, date1, nonDocValuesField, date2));
+    }
+    assertU(commit());
+    assertTrue(h.getCore().getLatestSchema().getField(docValuesField).hasDocValues());
+    assertTrue(h.getCore().getLatestSchema().getField(docValuesField).getType() instanceof PointField);
+    assertQ(req("q", "*:*", "fl", "id", "facet", "true", "facet.range", docValuesField, "facet.range.start", "1995-01-10T10:59:59Z-10DAYS",
+        "facet.range.end", "1995-01-10T10:59:59Z+20DAYS", "facet.range.gap", "+2DAYS"),
+        "//*[@numFound='10']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-10T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-12T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-14T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-16T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-18T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-20T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-22T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-24T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-26T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-28T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1994-12-31T10:59:59Z'][.='0']");
+
+    assertQ(req("q", "*:*", "fl", "id", "facet", "true", "facet.range", docValuesField, "facet.range.start", "1995-01-10T10:59:59Z-10DAYS",
+        "facet.range.end", "1995-01-10T10:59:59Z+20DAYS", "facet.range.gap", "+2DAYS", "facet.range.method", "dv"),
+        "//*[@numFound='10']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-10T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-12T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-14T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-16T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-18T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-20T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-22T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-24T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-26T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-28T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1994-12-31T10:59:59Z'][.='0']");
+
+    assertQ(req("q", "*:*", "fl", "id", "facet", "true", "facet.range", docValuesField, "facet.range.start", "1995-01-10T10:59:59Z",
+        "facet.range.end", "1995-01-10T10:59:59Z+20DAYS", "facet.range.gap", "+100DAYS"),
+        "//*[@numFound='10']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='1995-01-10T10:59:59Z'][.='10']");
+
+    assertFalse(h.getCore().getLatestSchema().getField(nonDocValuesField).hasDocValues());
+    assertTrue(h.getCore().getLatestSchema().getField(nonDocValuesField).getType() instanceof PointField);
+    // Range Faceting with method = filter should work
+    assertQ(req("q", "*:*", "fl", "id", "facet", "true", "facet.range", nonDocValuesField, "facet.range.start", "1995-01-10T10:59:59Z-10DAYS",
+        "facet.range.end", "1995-01-10T10:59:59Z+20DAYS", "facet.range.gap", "+2DAYS", "facet.range.method", "filter"),
+        "//*[@numFound='10']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-10T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-12T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-14T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-16T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-18T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-20T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-22T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-24T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-26T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-28T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1994-12-31T10:59:59Z'][.='0']");
+
+    // this should actually use filter method instead of dv
+    assertQ(req("q", "*:*", "fl", "id", "facet", "true", "facet.range", nonDocValuesField, "facet.range.start", "1995-01-10T10:59:59Z-10DAYS",
+        "facet.range.end", "1995-01-10T10:59:59Z+20DAYS", "facet.range.gap", "+2DAYS", "facet.range.method", "dv"),
+        "//*[@numFound='10']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-10T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-12T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-14T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-16T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-18T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-20T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-22T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-24T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-26T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1995-01-28T10:59:59Z'][.='2']",
+        "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='1994-12-31T10:59:59Z'][.='0']");
+  }
+
+  private void doTestDatePointFunctionQuery(String dvFieldName, String nonDvFieldName, String type) throws Exception {
+    String baseDate = "1995-01-10T10:59:10Z";
+    for (int i = 0; i < 10; i++) {
+      String date = String.format(Locale.ROOT, "%s+%dSECONDS", baseDate, i+1);
+      assertU(adoc("id", String.valueOf(i), dvFieldName, date, nonDvFieldName, date));
+    }
+    assertU(commit());
+    assertTrue(h.getCore().getLatestSchema().getField(dvFieldName).hasDocValues());
+    assertTrue(h.getCore().getLatestSchema().getField(dvFieldName).getType() instanceof PointField);
+    assertQ(req("q", "*:*", "fl", "id, " + dvFieldName, "sort", "product(-1," + dvFieldName + ") asc"),
+        "//*[@numFound='10']",
+        "//result/doc[1]/" + type + "[@name='" + dvFieldName + "'][.='1995-01-10T10:59:11Z']",
+        "//result/doc[2]/" + type + "[@name='" + dvFieldName + "'][.='1995-01-10T10:59:12Z']",
+        "//result/doc[3]/" + type + "[@name='" + dvFieldName + "'][.='1995-01-10T10:59:13Z']",
+        "//result/doc[10]/" + type + "[@name='" + dvFieldName + "'][.='1995-01-10T10:59:20Z']");
+
+    assertQ(req("q", "*:*", "fl", "id, " + dvFieldName + ", ms(" + dvFieldName + ","+baseDate+")"),
+        "//*[@numFound='10']",
+        "//result/doc[1]/float[@name='ms(" + dvFieldName + "," + baseDate + ")'][.='1000.0']",
+        "//result/doc[2]/float[@name='ms(" + dvFieldName + "," + baseDate + ")'][.='2000.0']",
+        "//result/doc[3]/float[@name='ms(" + dvFieldName + "," + baseDate + ")'][.='3000.0']",
+        "//result/doc[10]/float[@name='ms(" + dvFieldName + "," + baseDate + ")'][.='10000.0']");
+
+    assertQ(req("q", "*:*", "fl", "id, " + dvFieldName + ", field(" + dvFieldName + ")"),
+        "//*[@numFound='10']",
+        "//result/doc[1]/" + type + "[@name='field(" + dvFieldName + ")'][.='1995-01-10T10:59:11Z']",
+        "//result/doc[2]/" + type + "[@name='field(" + dvFieldName + ")'][.='1995-01-10T10:59:12Z']",
+        "//result/doc[3]/" + type + "[@name='field(" + dvFieldName + ")'][.='1995-01-10T10:59:13Z']",
+        "//result/doc[10]/" + type + "[@name='field(" + dvFieldName + ")'][.='1995-01-10T10:59:20Z']");
+
+    assertFalse(h.getCore().getLatestSchema().getField(nonDvFieldName).hasDocValues());
+    assertTrue(h.getCore().getLatestSchema().getField(nonDvFieldName).getType() instanceof PointField);
+
+    assertQEx("Expecting Exception",
+        "sort param could not be parsed as a query",
+        req("q", "*:*", "fl", "id, " + nonDvFieldName, "sort", "product(-1," + nonDvFieldName + ") asc"),
+        SolrException.ErrorCode.BAD_REQUEST);
+  }
+
+  private void testDatePointStats(String field, String dvField, String[] dates) {
+    for (int i = 0; i < dates.length; i++) {
+      assertU(adoc("id", String.valueOf(i), dvField, dates[i], field, dates[i]));
+    }
+    assertU(adoc("id", String.valueOf(dates.length)));
+    assertU(commit());
+    assertTrue(h.getCore().getLatestSchema().getField(dvField).hasDocValues());
+    assertTrue(h.getCore().getLatestSchema().getField(dvField).getType() instanceof PointField);
+    assertQ(req("q", "*:*", "fl", "id, " + dvField, "stats", "true", "stats.field", dvField),
+        "//*[@numFound='11']",
+        "//lst[@name='stats']/lst[@name='stats_fields']/lst[@name='" + dvField+ "']/date[@name='min'][.='" + dates[0] + "']",
+        "//lst[@name='stats']/lst[@name='stats_fields']/lst[@name='" + dvField+ "']/date[@name='max'][.='" + dates[dates.length-1] + "']",
+        "//lst[@name='stats']/lst[@name='stats_fields']/lst[@name='" + dvField+ "']/long[@name='count'][.='" + dates.length + "']",
+        "//lst[@name='stats']/lst[@name='stats_fields']/lst[@name='" + dvField+ "']/long[@name='missing'][.='1']");
+
+    assertFalse(h.getCore().getLatestSchema().getField(field).hasDocValues());
+    assertTrue(h.getCore().getLatestSchema().getField(field).getType() instanceof PointField);
+    assertQEx("Expecting Exception",
+        "Can't calculate stats on a PointField without docValues",
+        req("q", "*:*", "fl", "id, " + field, "stats", "true", "stats.field", field),
+        SolrException.ErrorCode.BAD_REQUEST);
+  }
+
+  private void testDatePointFieldsAtomicUpdates(String field, String type) throws Exception {
+    String date = "1995-01-10T10:59:10Z";
+    assertU(adoc(sdoc("id", "1", field, date)));
+    assertU(commit());
+
+    assertQ(req("q", "id:1"),
+        "//result/doc[1]/" + type + "[@name='" + field + "'][.='"+date+"']");
+
+    assertU(adoc(sdoc("id", "1", field, ImmutableMap.of("set", date+"+2DAYS"))));
+    assertU(commit());
+
+    assertQ(req("q", "id:1"),
+        "//result/doc[1]/" + type + "[@name='" + field + "'][.='1995-01-12T10:59:10Z']");
+  }
+
+  private void testMultiValuedDatePointFieldsAtomicUpdates(String field, String type) throws Exception {
+    String date1 = "1995-01-10T10:59:10Z";
+    String date2 = "1995-01-11T10:59:10Z";
+    String date3 = "1995-01-12T10:59:10Z";
+    assertU(adoc(sdoc("id", "1", field, date1)));
+    assertU(commit());
+
+    assertQ(req("q", "id:1"),
+        "//result/doc[1]/arr[@name='" + field + "']/" + type + "[.='"+date1+"']",
+        "count(//result/doc[1]/arr[@name='" + field + "']/" + type + ")=1");
+
+    assertU(adoc(sdoc("id", "1", field, ImmutableMap.of("add", date2))));
+    assertU(commit());
+
+    assertQ(req("q", "id:1"),
+        "//result/doc[1]/arr[@name='" + field + "']/" + type + "[.='"+date1+"']",
+        "//result/doc[1]/arr[@name='" + field + "']/" + type + "[.='"+date2+"']",
+        "count(//result/doc[1]/arr[@name='" + field + "']/" + type + ")=2");
+
+    assertU(adoc(sdoc("id", "1", field, ImmutableMap.of("remove", date1))));
+    assertU(commit());
+
+    assertQ(req("q", "id:1"),
+        "//result/doc[1]/arr[@name='" + field + "']/" + type + "[.='"+date2+"']",
+        "count(//result/doc[1]/arr[@name='" + field + "']/" + type + ")=1");
+
+    assertU(adoc(sdoc("id", "1", field, ImmutableMap.of("set", ImmutableList.of(date1, date2, date3)))));
+    assertU(commit());
+
+    assertQ(req("q", "id:1"),
+        "//result/doc[1]/arr[@name='" + field + "']/" + type + "[.='"+date1+"']",
+        "//result/doc[1]/arr[@name='" + field + "']/" + type + "[.='"+date2+"']",
+        "//result/doc[1]/arr[@name='" + field + "']/" + type + "[.='"+date3+"']",
+        "count(//result/doc[1]/arr[@name='" + field + "']/" + type + ")=3");
+
+    assertU(adoc(sdoc("id", "1", field, ImmutableMap.of("removeregex", ".*"))));
+    assertU(commit());
+
+    assertQ(req("q", "id:1"),
+        "count(//result/doc[1]/arr[@name='" + field + "']/" + type + ")=0");
+
+  }
+
+
 }


[21/43] lucene-solr:feature/autoscaling: added a test

Posted by sh...@apache.org.
added a test


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

Branch: refs/heads/feature/autoscaling
Commit: c680f45f2d8ca126e2783c371e07b46bd16234c6
Parents: c85aac2
Author: Noble Paul <no...@apache.org>
Authored: Thu Mar 9 14:41:42 2017 +1030
Committer: Noble Paul <no...@apache.org>
Committed: Thu Mar 9 14:41:42 2017 +1030

----------------------------------------------------------------------
 .../TestRuleBasedAuthorizationPlugin.java       | 23 ++++++++++++++++++++
 1 file changed, 23 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c680f45f/solr/core/src/test/org/apache/solr/security/TestRuleBasedAuthorizationPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/TestRuleBasedAuthorizationPlugin.java b/solr/core/src/test/org/apache/solr/security/TestRuleBasedAuthorizationPlugin.java
index 03656c5..4cdc555 100644
--- a/solr/core/src/test/org/apache/solr/security/TestRuleBasedAuthorizationPlugin.java
+++ b/solr/core/src/test/org/apache/solr/security/TestRuleBasedAuthorizationPlugin.java
@@ -305,6 +305,21 @@ public class TestRuleBasedAuthorizationPlugin extends SolrTestCaseJ4 {
         "handler", new DumpRequestHandler(),
         "params", new MapSolrParams(singletonMap("key", "VAL2")))
         , FORBIDDEN);
+
+    checkRules(makeMap("resource", "/update",
+        "userPrincipal", "solr",
+        "requestType", RequestType.UNKNOWN,
+        "collectionRequests", "go",
+        "handler", new UpdateRequestHandler(),
+        "params", new MapSolrParams(singletonMap("key", "VAL2")))
+        , FORBIDDEN, (Map<String, Object>) Utils.fromJSONString( "{user-role:{" +
+        "      admin:[admin_role]," +
+        "      update:[update_role]," +
+        "      solr:[read_role]}," +
+        "    permissions:[" +
+        "      {name:update, role:[admin_role,update_role]}," +
+        "      {name:read, role:[admin_role,update_role,read_role]}" +
+        "]}"));
   }
 
   public void testEditRules() throws IOException {
@@ -438,5 +453,13 @@ public class TestRuleBasedAuthorizationPlugin extends SolrTestCaseJ4 {
     }
   }
 
+static String testPerms = "{user-role:{" +
+    "      admin:[admin_role]," +
+    "      update:[update_role]," +
+    "      solr:[read_role]}," +
+    "    permissions:[" +
+    "      {name:update,role:[admin_role,update_role]}," +
+    "      {name:read,role:[admin_role,update_role,read_role]" +
+    "]}";
 
 }


[06/43] lucene-solr:feature/autoscaling: SOLR-10178, SOLR-10079: Force tests to always use NoMergePolicy, also assert that it was used

Posted by sh...@apache.org.
SOLR-10178, SOLR-10079: Force tests to always use NoMergePolicy, also assert that it was used


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

Branch: refs/heads/feature/autoscaling
Commit: 190f4b6b935d39d5c08b9a23a07c9c891d197312
Parents: 21559fe
Author: Ishan Chattopadhyaya <is...@apache.org>
Authored: Tue Mar 7 19:02:26 2017 +0530
Committer: Ishan Chattopadhyaya <is...@apache.org>
Committed: Tue Mar 7 19:02:26 2017 +0530

----------------------------------------------------------------------
 .../solr/update/TestInPlaceUpdatesDistrib.java   | 19 ++++++++++++++++++-
 1 file changed, 18 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/190f4b6b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java
index b107cbd..4c90bc6 100644
--- a/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java
+++ b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java
@@ -30,6 +30,8 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.NoMergePolicy;
 import org.apache.lucene.util.LuceneTestCase.Slow;
 import org.apache.lucene.util.TestUtil;
 import org.apache.solr.client.solrj.SolrClient;
@@ -55,6 +57,7 @@ import org.apache.solr.common.util.NamedList;
 import org.apache.solr.index.NoMergePolicyFactory;
 import org.apache.solr.update.processor.DistributedUpdateProcessor;
 import org.apache.solr.util.DefaultSolrThreadFactory;
+import org.apache.solr.util.RefCounted;
 import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.BeforeClass;
@@ -82,7 +85,11 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
     // we need consistent segments that aren't re-ordered on merge because we're
     // asserting inplace updates happen by checking the internal [docid]
     systemSetPropertySolrTestsMergePolicyFactory(NoMergePolicyFactory.class.getName());
-    
+
+    // HACK: Don't use a RandomMergePolicy, but only use the mergePolicyFactory that we've just set
+    System.setProperty(SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICYFACTORY, "true");
+    System.setProperty(SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICY, "false");
+
     initCore(configString, schemaString);
     
     // sanity check that autocommits are disabled
@@ -90,6 +97,16 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
     assertEquals(-1, h.getCore().getSolrConfig().getUpdateHandlerInfo().autoSoftCommmitMaxTime);
     assertEquals(-1, h.getCore().getSolrConfig().getUpdateHandlerInfo().autoCommmitMaxDocs);
     assertEquals(-1, h.getCore().getSolrConfig().getUpdateHandlerInfo().autoSoftCommmitMaxDocs);
+    
+    // assert that NoMergePolicy was chosen
+    RefCounted<IndexWriter> iw = h.getCore().getSolrCoreState().getIndexWriter(h.getCore());
+    try {
+      IndexWriter writer = iw.get();
+      assertTrue("Actual merge policy is: " + writer.getConfig().getMergePolicy(),
+          writer.getConfig().getMergePolicy() instanceof NoMergePolicy); 
+    } finally {
+      iw.decref();
+    }
   }
   
   @After


[09/43] lucene-solr:feature/autoscaling: doap entries for 6.4.2

Posted by sh...@apache.org.
doap entries for 6.4.2


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

Branch: refs/heads/feature/autoscaling
Commit: 57e8543bfd08533132d145985cadfcbdc5c12c36
Parents: 0d2c027
Author: Ishan Chattopadhyaya <is...@apache.org>
Authored: Wed Mar 8 00:43:01 2017 +0530
Committer: Ishan Chattopadhyaya <is...@apache.org>
Committed: Wed Mar 8 00:48:48 2017 +0530

----------------------------------------------------------------------
 dev-tools/doap/lucene.rdf | 7 +++++++
 dev-tools/doap/solr.rdf   | 7 +++++++
 2 files changed, 14 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/57e8543b/dev-tools/doap/lucene.rdf
----------------------------------------------------------------------
diff --git a/dev-tools/doap/lucene.rdf b/dev-tools/doap/lucene.rdf
index 4b57d6c..c1950a0 100644
--- a/dev-tools/doap/lucene.rdf
+++ b/dev-tools/doap/lucene.rdf
@@ -68,6 +68,13 @@
 
     <release>
       <Version>
+        <name>lucene-6.4.2</name>
+        <created>2017-03-07</created>
+        <revision>6.4.2</revision>
+      </Version>
+    </release>
+    <release>
+      <Version>
         <name>lucene-6.4.1</name>
         <created>2017-02-06</created>
         <revision>6.4.1</revision>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/57e8543b/dev-tools/doap/solr.rdf
----------------------------------------------------------------------
diff --git a/dev-tools/doap/solr.rdf b/dev-tools/doap/solr.rdf
index 47a6652..45efd08 100644
--- a/dev-tools/doap/solr.rdf
+++ b/dev-tools/doap/solr.rdf
@@ -68,6 +68,13 @@
 
     <release>
       <Version>
+        <name>solr-6.4.2</name>
+        <created>2017-03-07</created>
+        <revision>6.4.2</revision>
+      </Version>
+    </release>
+    <release>
+      <Version>
         <name>solr-6.4.1</name>
         <created>2017-02-06</created>
         <revision>6.4.1</revision>


[42/43] lucene-solr:feature/autoscaling: SOLR-10236: Remove FieldType.getNumericType() from master

Posted by sh...@apache.org.
SOLR-10236: Remove FieldType.getNumericType() from master


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

Branch: refs/heads/feature/autoscaling
Commit: abec54bd5722bc818fe46e111cf652cd7671db86
Parents: e3a0b42
Author: Tomas Fernandez Lobbe <tf...@apache.org>
Authored: Mon Mar 13 16:00:16 2017 -0700
Committer: Tomas Fernandez Lobbe <tf...@apache.org>
Committed: Mon Mar 13 16:00:16 2017 -0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                   |  4 ++++
 .../org/apache/solr/schema/DatePointField.java     | 11 -----------
 .../org/apache/solr/schema/DoublePointField.java   | 12 ------------
 .../src/java/org/apache/solr/schema/EnumField.java |  9 ---------
 .../src/java/org/apache/solr/schema/FieldType.java | 11 -----------
 .../org/apache/solr/schema/FloatPointField.java    | 13 -------------
 .../java/org/apache/solr/schema/IntPointField.java | 11 -----------
 .../org/apache/solr/schema/LongPointField.java     | 11 -----------
 .../solr/schema/SpatialPointVectorFieldType.java   |  7 -------
 .../src/java/org/apache/solr/schema/TrieField.java | 17 -----------------
 10 files changed, 4 insertions(+), 102 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/abec54bd/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 63424dd..80ecea8 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -50,6 +50,8 @@ Upgrading from Solr 6.x
   factors should be indexed in a separate field and combined with the query
   score using a function query.
 
+* Deprecated method getNumericType() has been removed from FieldType. Use getNumberType() instead
+
 New Features
 ----------------------
 * SOLR-9857, SOLR-9858: Collect aggregated metrics from nodes and shard leaders in overseer. (ab)
@@ -80,6 +82,8 @@ Optimizations
   (yonik)
 
 Other Changes
+* SOLR-10236: Removed FieldType.getNumericType(). Use getNumberType() instead. (Tom�s Fern�ndez L�bbe)
+
 ----------------------
 
 ==================  6.5.0 ==================

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/abec54bd/solr/core/src/java/org/apache/solr/schema/DatePointField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/DatePointField.java b/solr/core/src/java/org/apache/solr/schema/DatePointField.java
index b3517db..377f571 100644
--- a/solr/core/src/java/org/apache/solr/schema/DatePointField.java
+++ b/solr/core/src/java/org/apache/solr/schema/DatePointField.java
@@ -17,7 +17,6 @@
 
 package org.apache.solr.schema;
 
-import java.lang.invoke.MethodHandles;
 import java.time.Instant;
 import java.util.Collection;
 import java.util.Date;
@@ -25,7 +24,6 @@ import java.util.Date;
 import org.apache.lucene.document.LongPoint;
 import org.apache.lucene.document.StoredField;
 import org.apache.lucene.index.IndexableField;
-import org.apache.lucene.legacy.LegacyNumericType;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.valuesource.LongFieldSource;
 import org.apache.lucene.queries.function.valuesource.MultiValuedLongFieldSource;
@@ -39,13 +37,9 @@ import org.apache.lucene.util.mutable.MutableValueLong;
 import org.apache.solr.search.QParser;
 import org.apache.solr.uninverting.UninvertingReader;
 import org.apache.solr.util.DateMathParser;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 public class DatePointField extends PointField implements DateValueFieldType {
 
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
   public DatePointField() {
     type = NumberType.DATE;
   }
@@ -165,11 +159,6 @@ public class DatePointField extends PointField implements DateValueFieldType {
   }
 
   @Override
-  public LegacyNumericType getNumericType() {
-    return LegacyNumericType.LONG;
-  }
-
-  @Override
   public IndexableField createField(SchemaField field, Object value) {
     if (!isFieldUsed(field)) return null;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/abec54bd/solr/core/src/java/org/apache/solr/schema/DoublePointField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/DoublePointField.java b/solr/core/src/java/org/apache/solr/schema/DoublePointField.java
index e34ebec..6ae8349 100644
--- a/solr/core/src/java/org/apache/solr/schema/DoublePointField.java
+++ b/solr/core/src/java/org/apache/solr/schema/DoublePointField.java
@@ -17,14 +17,12 @@
 
 package org.apache.solr.schema;
 
-import java.lang.invoke.MethodHandles;
 import java.util.Collection;
 
 import org.apache.lucene.document.DoublePoint;
 import org.apache.lucene.document.StoredField;
 import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.index.IndexableField;
-import org.apache.lucene.legacy.LegacyNumericType;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.valuesource.DoubleFieldSource;
 import org.apache.lucene.queries.function.valuesource.MultiValuedDoubleFieldSource;
@@ -36,8 +34,6 @@ import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.NumericUtils;
 import org.apache.solr.search.QParser;
 import org.apache.solr.uninverting.UninvertingReader.Type;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * {@code PointField} implementation for {@code Double} values.
@@ -46,8 +42,6 @@ import org.slf4j.LoggerFactory;
  */
 public class DoublePointField extends PointField implements DoubleValueFieldType {
 
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
   public DoublePointField() {
     type = NumberType.DOUBLE;
   }
@@ -172,12 +166,6 @@ public class DoublePointField extends PointField implements DoubleValueFieldType
   }
 
   @Override
-  public LegacyNumericType getNumericType() {
-    // TODO: refactor this to not use LegacyNumericType
-    return LegacyNumericType.DOUBLE;
-  }
-
-  @Override
   public IndexableField createField(SchemaField field, Object value) {
     if (!isFieldUsed(field)) return null;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/abec54bd/solr/core/src/java/org/apache/solr/schema/EnumField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/EnumField.java b/solr/core/src/java/org/apache/solr/schema/EnumField.java
index 3e83db4..2e73f74 100644
--- a/solr/core/src/java/org/apache/solr/schema/EnumField.java
+++ b/solr/core/src/java/org/apache/solr/schema/EnumField.java
@@ -233,15 +233,6 @@ public class EnumField extends PrimitiveFieldType {
    * {@inheritDoc}
    */
   @Override
-  @Deprecated
-  public LegacyNumericType getNumericType() {
-    return LegacyNumericType.INT;
-  }
-  
-  /**
-   * {@inheritDoc}
-   */
-  @Override
   public NumberType getNumberType() {
     return NumberType.INTEGER;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/abec54bd/solr/core/src/java/org/apache/solr/schema/FieldType.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/FieldType.java b/solr/core/src/java/org/apache/solr/schema/FieldType.java
index c542a95..67b7be7 100644
--- a/solr/core/src/java/org/apache/solr/schema/FieldType.java
+++ b/solr/core/src/java/org/apache/solr/schema/FieldType.java
@@ -39,7 +39,6 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.document.SortedSetDocValuesField;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.legacy.LegacyNumericType;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
@@ -610,16 +609,6 @@ public abstract class FieldType extends FieldProperties {
     return similarityFactory;
   }
 
-
-  /** Return the numeric type of this field, or null if this field is not a
-   *  numeric field. 
-   *  @deprecated Please use {@link FieldType#getNumberType()} instead
-   */
-  @Deprecated
-  public LegacyNumericType getNumericType() {
-    return null;
-  }
-  
   /**
    * Return the numeric type of this field, or null if this field is not a
    * numeric field. 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/abec54bd/solr/core/src/java/org/apache/solr/schema/FloatPointField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/FloatPointField.java b/solr/core/src/java/org/apache/solr/schema/FloatPointField.java
index 39453e7..0f42cfd 100644
--- a/solr/core/src/java/org/apache/solr/schema/FloatPointField.java
+++ b/solr/core/src/java/org/apache/solr/schema/FloatPointField.java
@@ -17,14 +17,12 @@
 
 package org.apache.solr.schema;
 
-import java.lang.invoke.MethodHandles;
 import java.util.Collection;
 
 import org.apache.lucene.document.FloatPoint;
 import org.apache.lucene.document.StoredField;
 import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.index.IndexableField;
-import org.apache.lucene.legacy.LegacyNumericType;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.valuesource.FloatFieldSource;
 import org.apache.lucene.queries.function.valuesource.MultiValuedFloatFieldSource;
@@ -36,8 +34,6 @@ import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.NumericUtils;
 import org.apache.solr.search.QParser;
 import org.apache.solr.uninverting.UninvertingReader.Type;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * {@code PointField} implementation for {@code Float} values.
@@ -46,8 +42,6 @@ import org.slf4j.LoggerFactory;
  */
 public class FloatPointField extends PointField implements FloatValueFieldType {
 
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
   public FloatPointField() {
     type = NumberType.FLOAT;
   }
@@ -171,13 +165,6 @@ public class FloatPointField extends PointField implements FloatValueFieldType {
     return new MultiValuedFloatFieldSource(f.getName(), choice);
   }
 
-
-  @Override
-  public LegacyNumericType getNumericType() {
-    // TODO: refactor this to not use LegacyNumericType
-    return LegacyNumericType.FLOAT;
-  }
-
   @Override
   public IndexableField createField(SchemaField field, Object value) {
     if (!isFieldUsed(field)) return null;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/abec54bd/solr/core/src/java/org/apache/solr/schema/IntPointField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/IntPointField.java b/solr/core/src/java/org/apache/solr/schema/IntPointField.java
index db26988..5eaf7e0 100644
--- a/solr/core/src/java/org/apache/solr/schema/IntPointField.java
+++ b/solr/core/src/java/org/apache/solr/schema/IntPointField.java
@@ -17,13 +17,11 @@
 
 package org.apache.solr.schema;
 
-import java.lang.invoke.MethodHandles;
 import java.util.Collection;
 
 import org.apache.lucene.document.IntPoint;
 import org.apache.lucene.document.StoredField;
 import org.apache.lucene.index.IndexableField;
-import org.apache.lucene.legacy.LegacyNumericType;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.valuesource.IntFieldSource;
 import org.apache.lucene.queries.function.valuesource.MultiValuedIntFieldSource;
@@ -34,8 +32,6 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.solr.search.QParser;
 import org.apache.solr.uninverting.UninvertingReader.Type;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * {@code PointField} implementation for {@code Integer} values.
@@ -44,8 +40,6 @@ import org.slf4j.LoggerFactory;
  */
 public class IntPointField extends PointField implements IntValueFieldType {
 
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
   public IntPointField() {
     type = NumberType.INTEGER;
   }
@@ -164,11 +158,6 @@ public class IntPointField extends PointField implements IntValueFieldType {
   }
 
   @Override
-  public LegacyNumericType getNumericType() {
-    return LegacyNumericType.INT;
-  }
-
-  @Override
   public IndexableField createField(SchemaField field, Object value) {
     if (!isFieldUsed(field)) return null;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/abec54bd/solr/core/src/java/org/apache/solr/schema/LongPointField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/LongPointField.java b/solr/core/src/java/org/apache/solr/schema/LongPointField.java
index f5d0948..e58fbcf 100644
--- a/solr/core/src/java/org/apache/solr/schema/LongPointField.java
+++ b/solr/core/src/java/org/apache/solr/schema/LongPointField.java
@@ -17,13 +17,11 @@
 
 package org.apache.solr.schema;
 
-import java.lang.invoke.MethodHandles;
 import java.util.Collection;
 
 import org.apache.lucene.document.LongPoint;
 import org.apache.lucene.document.StoredField;
 import org.apache.lucene.index.IndexableField;
-import org.apache.lucene.legacy.LegacyNumericType;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.valuesource.LongFieldSource;
 import org.apache.lucene.queries.function.valuesource.MultiValuedLongFieldSource;
@@ -33,8 +31,6 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.solr.search.QParser;
 import org.apache.solr.uninverting.UninvertingReader.Type;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * {@code PointField} implementation for {@code Long} values.
@@ -43,8 +39,6 @@ import org.slf4j.LoggerFactory;
  */
 public class LongPointField extends PointField implements LongValueFieldType {
 
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
   public LongPointField() {
     type = NumberType.LONG;
   }
@@ -169,11 +163,6 @@ public class LongPointField extends PointField implements LongValueFieldType {
   }
 
   @Override
-  public LegacyNumericType getNumericType() {
-    return LegacyNumericType.LONG;
-  }
-
-  @Override
   public IndexableField createField(SchemaField field, Object value) {
     if (!isFieldUsed(field)) return null;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/abec54bd/solr/core/src/java/org/apache/solr/schema/SpatialPointVectorFieldType.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/SpatialPointVectorFieldType.java b/solr/core/src/java/org/apache/solr/schema/SpatialPointVectorFieldType.java
index b4b3d2b..5c7734e 100644
--- a/solr/core/src/java/org/apache/solr/schema/SpatialPointVectorFieldType.java
+++ b/solr/core/src/java/org/apache/solr/schema/SpatialPointVectorFieldType.java
@@ -21,7 +21,6 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.lucene.legacy.LegacyFieldType;
-import org.apache.lucene.legacy.LegacyNumericType;
 import org.apache.lucene.spatial.vector.PointVectorStrategy;
 
 /**
@@ -80,12 +79,6 @@ public class SpatialPointVectorFieldType extends AbstractSpatialFieldType<PointV
   }
 
   @Override
-  @Deprecated
-  public LegacyNumericType getNumericType() {
-    return LegacyNumericType.DOUBLE;
-  }
-  
-  @Override
   public NumberType getNumberType() {
     return NumberType.DOUBLE;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/abec54bd/solr/core/src/java/org/apache/solr/schema/TrieField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/TrieField.java b/solr/core/src/java/org/apache/solr/schema/TrieField.java
index b70b2b0..46240c9 100644
--- a/solr/core/src/java/org/apache/solr/schema/TrieField.java
+++ b/solr/core/src/java/org/apache/solr/schema/TrieField.java
@@ -337,23 +337,6 @@ public class TrieField extends NumericFieldType {
   }
 
   @Override
-  public LegacyNumericType getNumericType() {
-    switch (type) {
-      case INTEGER:
-        return LegacyNumericType.INT;
-      case LONG:
-      case DATE:
-        return LegacyNumericType.LONG;
-      case FLOAT:
-        return LegacyNumericType.FLOAT;
-      case DOUBLE:
-        return LegacyNumericType.DOUBLE;
-      default:
-        throw new AssertionError();
-    }
-  }
-
-  @Override
   public Query getRangeQuery(QParser parser, SchemaField field, String min, String max, boolean minInclusive, boolean maxInclusive) {
     if (field.multiValued() && field.hasDocValues() && !field.indexed()) {
       // for the multi-valued dv-case, the default rangeimpl over toInternal is correct


[07/43] lucene-solr:feature/autoscaling: SOLR-10226 JMX metric avgTimePerRequest broken.

Posted by sh...@apache.org.
SOLR-10226 JMX metric avgTimePerRequest broken.


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

Branch: refs/heads/feature/autoscaling
Commit: 2d51a42d3cae3eddc89f407cd3611fa2cd5d55d0
Parents: 190f4b6
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Tue Mar 7 17:59:57 2017 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Tue Mar 7 18:09:58 2017 +0100

----------------------------------------------------------------------
 solr/CHANGES.txt                                              | 6 ++++++
 .../src/java/org/apache/solr/handler/RequestHandlerBase.java  | 7 ++++++-
 .../core/src/java/org/apache/solr/util/stats/MetricUtils.java | 2 +-
 3 files changed, 13 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2d51a42d/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index db721da..5b0eb03 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -96,6 +96,10 @@ Detailed Change List
 
 Upgrade Notes
 ----------------------
+* SOLR-10226: JMX metric "avgTimePerRequest" (and the corresponding metric in the metrics API for
+  each handler) used to be a simple non-decaying average based on total cumulative time and the
+  number of requests. New Codahale Metrics implementation applies exponential decay to this value,
+  which heavily biases the average towards the last 5 minutes. (ab)
 
 New Features
 ----------------------
@@ -212,6 +216,8 @@ Bug Fixes
 
 * SOLR-10088: Installer script does not put zoo.cfg in SOLR_HOME (janhoy)
 
+* SOLR-10226: add back "totalTime" metric to all handlers. See also the back-compat note. (ab)
+
 Optimizations
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2d51a42d/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 3c6f5fa..1958e11 100644
--- a/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
+++ b/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
@@ -66,6 +66,7 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
   private Meter numTimeouts = new Meter();
   private Counter requests = new Counter();
   private Timer requestTimes = new Timer();
+  private Counter totalTime = new Counter();
 
   private final long handlerStart;
 
@@ -143,6 +144,7 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
     numTimeouts = manager.meter(registryName, "timeouts", getCategory().toString(), scope);
     requests = manager.counter(registryName, "requests", getCategory().toString(), scope);
     requestTimes = manager.timer(registryName, "requestTimes", getCategory().toString(), scope);
+    totalTime = manager.counter(registryName, "totalTime", getCategory().toString(), scope);
   }
 
   public static SolrParams getSolrParamsFromNamedList(NamedList args, String key) {
@@ -209,7 +211,8 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
         }
       }
     } finally {
-      timer.stop();
+      long elapsed = timer.stop();
+      totalTime.inc(elapsed);
     }
   }
 
@@ -292,6 +295,8 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
     lst.add("serverErrors", numServerErrors.getCount());
     lst.add("clientErrors", numClientErrors.getCount());
     lst.add("timeouts", numTimeouts.getCount());
+    // convert totalTime to ms
+    lst.add("totalTime", MetricUtils.nsToMs(totalTime.getCount()));
     MetricUtils.addMetrics(lst, requestTimes);
     return lst;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2d51a42d/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java b/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java
index 0d386ae..80f035b 100644
--- a/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java
+++ b/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java
@@ -63,7 +63,7 @@ public class MetricUtils {
    * @param ns the amount of time in nanoseconds
    * @return the amount of time in milliseconds
    */
-  static double nsToMs(double ns) {
+  public static double nsToMs(double ns) {
     return ns / TimeUnit.MILLISECONDS.toNanos(1);
   }
 


[23/43] lucene-solr:feature/autoscaling: LUCENE-7734: FieldType copy constructor widened to IndexableFieldType

Posted by sh...@apache.org.
LUCENE-7734: FieldType copy constructor widened to IndexableFieldType


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

Branch: refs/heads/feature/autoscaling
Commit: d2bf30d58fbfc9279bed663500400153b4d4df44
Parents: d945a24
Author: David Smiley <ds...@apache.org>
Authored: Thu Mar 9 23:12:45 2017 -0500
Committer: David Smiley <ds...@apache.org>
Committed: Thu Mar 9 23:12:45 2017 -0500

----------------------------------------------------------------------
 lucene/CHANGES.txt                                             | 3 +++
 lucene/core/src/java/org/apache/lucene/document/FieldType.java | 6 +++---
 2 files changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d2bf30d5/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index a8f7ee4..4040945 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -117,6 +117,9 @@ API Changes
   instead of once all shard responses are present. (Simon Willnauer,
   Mike McCandless)
 
+* LUCENE-7734: FieldType's copy constructor was widened to accept any IndexableFieldType.
+  (David Smiley)
+
 New Features
 
 * LUCENE-7449: Add CROSSES relation support to RangeFieldQuery. (Nick Knize)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d2bf30d5/lucene/core/src/java/org/apache/lucene/document/FieldType.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/FieldType.java b/lucene/core/src/java/org/apache/lucene/document/FieldType.java
index 6f206a4..3c7d276 100644
--- a/lucene/core/src/java/org/apache/lucene/document/FieldType.java
+++ b/lucene/core/src/java/org/apache/lucene/document/FieldType.java
@@ -44,7 +44,7 @@ public class FieldType implements IndexableFieldType  {
   /**
    * Create a new mutable FieldType with all of the properties from <code>ref</code>
    */
-  public FieldType(FieldType ref) {
+  public FieldType(IndexableFieldType ref) {
     this.stored = ref.stored();
     this.tokenized = ref.tokenized();
     this.storeTermVectors = ref.storeTermVectors();
@@ -54,8 +54,8 @@ public class FieldType implements IndexableFieldType  {
     this.omitNorms = ref.omitNorms();
     this.indexOptions = ref.indexOptions();
     this.docValuesType = ref.docValuesType();
-    this.dimensionCount = ref.dimensionCount;
-    this.dimensionNumBytes = ref.dimensionNumBytes;
+    this.dimensionCount = ref.pointDimensionCount();
+    this.dimensionNumBytes = ref.pointNumBytes();
     // Do not copy frozen!
   }
   


[32/43] lucene-solr:feature/autoscaling: LUCENE-7740: Refactor Range Fields to remove Field suffix (e.g., DoubleRange), move InetAddressRange and InetAddressPoint from sandbox to misc module, and refactor all other range fields from sandbox to core.

Posted by sh...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressPoint.java b/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressPoint.java
deleted file mode 100644
index 5cda742..0000000
--- a/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressPoint.java
+++ /dev/null
@@ -1,313 +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.lucene.document;
-
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.Arrays;
-import java.util.Comparator;
-
-import org.apache.lucene.index.PointValues;
-import org.apache.lucene.search.PointInSetQuery;
-import org.apache.lucene.search.PointRangeQuery;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.NumericUtils;
-import org.apache.lucene.util.StringHelper;
-
-/** 
- * An indexed 128-bit {@code InetAddress} field.
- * <p>
- * Finding all documents within a range at search time is
- * efficient.  Multiple values for the same field in one document
- * is allowed. 
- * <p>
- * This field defines static factory methods for creating common queries:
- * <ul>
- *   <li>{@link #newExactQuery(String, InetAddress)} for matching an exact network address.
- *   <li>{@link #newPrefixQuery(String, InetAddress, int)} for matching a network based on CIDR prefix.
- *   <li>{@link #newRangeQuery(String, InetAddress, InetAddress)} for matching arbitrary network address ranges.
- *   <li>{@link #newSetQuery(String, InetAddress...)} for matching a set of network addresses.
- * </ul>
- * <p>
- * This field supports both IPv4 and IPv6 addresses: IPv4 addresses are converted
- * to <a href="https://tools.ietf.org/html/rfc4291#section-2.5.5">IPv4-Mapped IPv6 Addresses</a>:
- * indexing {@code 1.2.3.4} is the same as indexing {@code ::FFFF:1.2.3.4}.
- * @see PointValues
- */
-public class InetAddressPoint extends Field {
-
-  // implementation note: we convert all addresses to IPv6: we expect prefix compression of values,
-  // so its not wasteful, but allows one field to handle both IPv4 and IPv6.
-  /** The number of bytes per dimension: 128 bits */
-  public static final int BYTES = 16;
-  
-  // rfc4291 prefix
-  static final byte[] IPV4_PREFIX = new byte[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, -1, -1 }; 
-
-  private static final FieldType TYPE;
-  static {
-    TYPE = new FieldType();
-    TYPE.setDimensions(1, BYTES);
-    TYPE.freeze();
-  }
-
-  /** The minimum value that an ip address can hold. */
-  public static final InetAddress MIN_VALUE;
-  /** The maximum value that an ip address can hold. */
-  public static final InetAddress MAX_VALUE;
-  static {
-    MIN_VALUE = decode(new byte[BYTES]);
-    byte[] maxValueBytes = new byte[BYTES];
-    Arrays.fill(maxValueBytes, (byte) 0xFF);
-    MAX_VALUE = decode(maxValueBytes);
-  }
-
-  /**
-   * Return the {@link InetAddress} that compares immediately greater than
-   * {@code address}.
-   * @throws ArithmeticException if the provided address is the
-   *              {@link #MAX_VALUE maximum ip address}
-   */
-  public static InetAddress nextUp(InetAddress address) {
-    if (address.equals(MAX_VALUE)) {
-      throw new ArithmeticException("Overflow: there is no greater InetAddress than "
-          + address.getHostAddress());
-    }
-    byte[] delta = new byte[BYTES];
-    delta[BYTES-1] = 1;
-    byte[] nextUpBytes = new byte[InetAddressPoint.BYTES];
-    NumericUtils.add(InetAddressPoint.BYTES, 0, encode(address), delta, nextUpBytes);
-    return decode(nextUpBytes);
-  }
-
-  /**
-   * Return the {@link InetAddress} that compares immediately less than
-   * {@code address}.
-   * @throws ArithmeticException if the provided address is the
-   *              {@link #MIN_VALUE minimum ip address}
-   */
-  public static InetAddress nextDown(InetAddress address) {
-    if (address.equals(MIN_VALUE)) {
-      throw new ArithmeticException("Underflow: there is no smaller InetAddress than "
-          + address.getHostAddress());
-    }
-    byte[] delta = new byte[BYTES];
-    delta[BYTES-1] = 1;
-    byte[] nextDownBytes = new byte[InetAddressPoint.BYTES];
-    NumericUtils.subtract(InetAddressPoint.BYTES, 0, encode(address), delta, nextDownBytes);
-    return decode(nextDownBytes);
-  }
-
-  /** Change the values of this field */
-  public void setInetAddressValue(InetAddress value) {
-    if (value == null) {
-      throw new IllegalArgumentException("point must not be null");
-    }
-    fieldsData = new BytesRef(encode(value));
-  }
-
-  @Override
-  public void setBytesValue(BytesRef bytes) {
-    throw new IllegalArgumentException("cannot change value type from InetAddress to BytesRef");
-  }
-
-  /** Creates a new InetAddressPoint, indexing the
-   *  provided address.
-   *
-   *  @param name field name
-   *  @param point InetAddress value
-   *  @throws IllegalArgumentException if the field name or value is null.
-   */
-  public InetAddressPoint(String name, InetAddress point) {
-    super(name, TYPE);
-    setInetAddressValue(point);
-  }
-  
-  @Override
-  public String toString() {
-    StringBuilder result = new StringBuilder();
-    result.append(getClass().getSimpleName());
-    result.append(" <");
-    result.append(name);
-    result.append(':');
-
-    // IPv6 addresses are bracketed, to not cause confusion with historic field:value representation
-    BytesRef bytes = (BytesRef) fieldsData;
-    InetAddress address = decode(BytesRef.deepCopyOf(bytes).bytes);
-    if (address.getAddress().length == 16) {
-      result.append('[');
-      result.append(address.getHostAddress());
-      result.append(']');
-    } else {
-      result.append(address.getHostAddress());
-    }
-
-    result.append('>');
-    return result.toString();
-  }
-  
-  // public helper methods (e.g. for queries)
-
-  /** Encode InetAddress value into binary encoding */
-  public static byte[] encode(InetAddress value) {
-    byte[] address = value.getAddress();
-    if (address.length == 4) {
-      byte[] mapped = new byte[16];
-      System.arraycopy(IPV4_PREFIX, 0, mapped, 0, IPV4_PREFIX.length);
-      System.arraycopy(address, 0, mapped, IPV4_PREFIX.length, address.length);
-      address = mapped;
-    } else if (address.length != 16) {
-      // more of an assertion, how did you create such an InetAddress :)
-      throw new UnsupportedOperationException("Only IPv4 and IPv6 addresses are supported");
-    }
-    return address;
-  }
-  
-  /** Decodes InetAddress value from binary encoding */
-  public static InetAddress decode(byte value[]) {
-    try {
-      return InetAddress.getByAddress(value);
-    } catch (UnknownHostException e) {
-      // this only happens if value.length != 4 or 16, strange exception class
-      throw new IllegalArgumentException("encoded bytes are of incorrect length", e);
-    }
-  }
-
-  // static methods for generating queries
-
-  /** 
-   * Create a query for matching a network address.
-   *
-   * @param field field name. must not be {@code null}.
-   * @param value exact value
-   * @throws IllegalArgumentException if {@code field} is null.
-   * @return a query matching documents with this exact value
-   */
-  public static Query newExactQuery(String field, InetAddress value) {
-    return newRangeQuery(field, value, value);
-  }
-  
-  /** 
-   * Create a prefix query for matching a CIDR network range.
-   *
-   * @param field field name. must not be {@code null}.
-   * @param value any host address
-   * @param prefixLength the network prefix length for this address. This is also known as the subnet mask in the context of IPv4 addresses.
-   * @throws IllegalArgumentException if {@code field} is null, or prefixLength is invalid.
-   * @return a query matching documents with addresses contained within this network
-   */
-  public static Query newPrefixQuery(String field, InetAddress value, int prefixLength) {
-    if (value == null) {
-      throw new IllegalArgumentException("InetAddress must not be null");
-    }
-    if (prefixLength < 0 || prefixLength > 8 * value.getAddress().length) {
-      throw new IllegalArgumentException("illegal prefixLength '" + prefixLength + "'. Must be 0-32 for IPv4 ranges, 0-128 for IPv6 ranges");
-    }
-    // create the lower value by zeroing out the host portion, upper value by filling it with all ones.
-    byte lower[] = value.getAddress();
-    byte upper[] = value.getAddress();
-    for (int i = prefixLength; i < 8 * lower.length; i++) {
-      int m = 1 << (7 - (i & 7));
-      lower[i >> 3] &= ~m;
-      upper[i >> 3] |= m;
-    }
-    try {
-      return newRangeQuery(field, InetAddress.getByAddress(lower), InetAddress.getByAddress(upper));
-    } catch (UnknownHostException e) {
-      throw new AssertionError(e); // values are coming from InetAddress
-    }
-  }
-
-  /** 
-   * Create a range query for network addresses.
-   * <p>
-   * You can have half-open ranges (which are in fact &lt;/&le; or &gt;/&ge; queries)
-   * by setting {@code lowerValue = InetAddressPoint.MIN_VALUE} or
-   * {@code upperValue = InetAddressPoint.MAX_VALUE}.
-   * <p> Ranges are inclusive. For exclusive ranges, pass {@code InetAddressPoint#nextUp(lowerValue)}
-   * or {@code InetAddressPoint#nexDown(upperValue)}.
-   *
-   * @param field field name. must not be {@code null}.
-   * @param lowerValue lower portion of the range (inclusive). must not be null.
-   * @param upperValue upper portion of the range (inclusive). must not be null.
-   * @throws IllegalArgumentException if {@code field} is null, {@code lowerValue} is null, 
-   *                                  or {@code upperValue} is null
-   * @return a query matching documents within this range.
-   */
-  public static Query newRangeQuery(String field, InetAddress lowerValue, InetAddress upperValue) {
-    PointRangeQuery.checkArgs(field, lowerValue, upperValue);
-    return new PointRangeQuery(field, encode(lowerValue), encode(upperValue), 1) {
-      @Override
-      protected String toString(int dimension, byte[] value) {
-        return decode(value).getHostAddress(); // for ranges, the range itself is already bracketed
-      }
-    };
-  }
-
-  /**
-   * Create a query matching any of the specified 1D values.  This is the points equivalent of {@code TermsQuery}.
-   * 
-   * @param field field name. must not be {@code null}.
-   * @param values all values to match
-   */
-  public static Query newSetQuery(String field, InetAddress... values) {
-
-    // We must compare the encoded form (InetAddress doesn't implement Comparable, and even if it
-    // did, we do our own thing with ipv4 addresses):
-
-    // NOTE: we could instead convert-per-comparison and save this extra array, at cost of slower sort:
-    byte[][] sortedValues = new byte[values.length][];
-    for(int i=0;i<values.length;i++) {
-      sortedValues[i] = encode(values[i]);
-    }
-
-    Arrays.sort(sortedValues,
-                new Comparator<byte[]>() {
-                  @Override
-                  public int compare(byte[] a, byte[] b) {
-                    return StringHelper.compare(BYTES, a, 0, b, 0);
-                  }
-                });
-
-    final BytesRef encoded = new BytesRef(new byte[BYTES]);
-
-    return new PointInSetQuery(field, 1, BYTES,
-                               new PointInSetQuery.Stream() {
-
-                                 int upto;
-
-                                 @Override
-                                 public BytesRef next() {
-                                   if (upto == sortedValues.length) {
-                                     return null;
-                                   } else {
-                                     encoded.bytes = sortedValues[upto];
-                                     assert encoded.bytes.length == encoded.length;
-                                     upto++;
-                                     return encoded;
-                                   }
-                                 }
-                               }) {
-      @Override
-      protected String toString(byte[] value) {
-        assert value.length == BYTES;
-        return decode(value).getHostAddress();
-      }
-    };
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressRangeField.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressRangeField.java b/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressRangeField.java
deleted file mode 100644
index c6ebc83..0000000
--- a/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressRangeField.java
+++ /dev/null
@@ -1,168 +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.lucene.document;
-
-import java.net.InetAddress;
-
-import org.apache.lucene.document.RangeFieldQuery.QueryType;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.StringHelper;
-
-/**
- * An indexed InetAddress Range Field
- * <p>
- * This field indexes an {@code InetAddress} range defined as a min/max pairs. It is single
- * dimension only (indexed as two 16 byte paired values).
- * <p>
- * Multiple values are supported.
- *
- * <p>
- * This field defines the following static factory methods for common search operations over Ip Ranges
- * <ul>
- *   <li>{@link #newIntersectsQuery newIntersectsQuery()} matches ip ranges that intersect the defined search range.
- *   <li>{@link #newWithinQuery newWithinQuery()} matches ip ranges that are within the defined search range.
- *   <li>{@link #newContainsQuery newContainsQuery()} matches ip ranges that contain the defined search range.
- *   <li>{@link #newCrossesQuery newCrossesQuery()} matches ip ranges that cross the defined search range
- * </ul>
- */
-public class InetAddressRangeField extends Field {
-  /** The number of bytes per dimension : sync w/ {@code InetAddressPoint} */
-  public static final int BYTES = InetAddressPoint.BYTES;
-
-  private static final FieldType TYPE;
-  static {
-    TYPE = new FieldType();
-    TYPE.setDimensions(2, BYTES);
-    TYPE.freeze();
-  }
-
-  /**
-   * Create a new InetAddressRangeField from min/max value
-   * @param name field name. must not be null.
-   * @param min range min value; defined as an {@code InetAddress}
-   * @param max range max value; defined as an {@code InetAddress}
-   */
-  public InetAddressRangeField(String name, final InetAddress min, final InetAddress max) {
-    super(name, TYPE);
-    setRangeValues(min, max);
-  }
-
-  /**
-   * Change (or set) the min/max values of the field.
-   * @param min range min value; defined as an {@code InetAddress}
-   * @param max range max value; defined as an {@code InetAddress}
-   */
-  public void setRangeValues(InetAddress min, InetAddress max) {
-    if (StringHelper.compare(BYTES, min.getAddress(), 0, max.getAddress(), 0) > 0) {
-      throw new IllegalArgumentException("min value cannot be greater than max value for range field (name=" + name + ")");
-    }
-    final byte[] bytes;
-    if (fieldsData == null) {
-      bytes = new byte[BYTES*2];
-      fieldsData = new BytesRef(bytes);
-    } else {
-      bytes = ((BytesRef)fieldsData).bytes;
-    }
-    encode(min, max, bytes);
-  }
-
-  /** encode the min/max range into the provided byte array */
-  private static void encode(final InetAddress min, final InetAddress max, final byte[] bytes) {
-    System.arraycopy(InetAddressPoint.encode(min), 0, bytes, 0, BYTES);
-    System.arraycopy(InetAddressPoint.encode(max), 0, bytes, BYTES, BYTES);
-  }
-
-  /** encode the min/max range and return the byte array */
-  private static byte[] encode(InetAddress min, InetAddress max) {
-    byte[] b = new byte[BYTES*2];
-    encode(min, max, b);
-    return b;
-  }
-
-  /**
-   * Create a query for matching indexed ip ranges that {@code INTERSECT} the defined range.
-   * @param field field name. must not be null.
-   * @param min range min value; provided as an {@code InetAddress}
-   * @param max range max value; provided as an {@code InetAddress}
-   * @return query for matching intersecting ranges (overlap, within, crosses, or contains)
-   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
-   */
-  public static Query newIntersectsQuery(String field, final InetAddress min, final InetAddress max) {
-    return newRelationQuery(field, min, max, QueryType.INTERSECTS);
-  }
-
-  /**
-   * Create a query for matching indexed ip ranges that {@code CONTAINS} the defined range.
-   * @param field field name. must not be null.
-   * @param min range min value; provided as an {@code InetAddress}
-   * @param max range max value; provided as an {@code InetAddress}
-   * @return query for matching intersecting ranges (overlap, within, crosses, or contains)
-   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
-   */
-  public static Query newContainsQuery(String field, final InetAddress min, final InetAddress max) {
-    return newRelationQuery(field, min, max, QueryType.CONTAINS);
-  }
-
-  /**
-   * Create a query for matching indexed ip ranges that are {@code WITHIN} the defined range.
-   * @param field field name. must not be null.
-   * @param min range min value; provided as an {@code InetAddress}
-   * @param max range max value; provided as an {@code InetAddress}
-   * @return query for matching intersecting ranges (overlap, within, crosses, or contains)
-   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
-   */
-  public static Query newWithinQuery(String field, final InetAddress min, final InetAddress max) {
-    return newRelationQuery(field, min, max, QueryType.WITHIN);
-  }
-
-  /**
-   * Create a query for matching indexed ip ranges that {@code CROSS} the defined range.
-   * @param field field name. must not be null.
-   * @param min range min value; provided as an {@code InetAddress}
-   * @param max range max value; provided as an {@code InetAddress}
-   * @return query for matching intersecting ranges (overlap, within, crosses, or contains)
-   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
-   */
-  public static Query newCrossesQuery(String field, final InetAddress min, final InetAddress max) {
-    return newRelationQuery(field, min, max, QueryType.CROSSES);
-  }
-
-  /** helper method for creating the desired relational query */
-  private static Query newRelationQuery(String field, final InetAddress min, final InetAddress max, QueryType relation) {
-    return new RangeFieldQuery(field, encode(min, max), 1, relation) {
-      @Override
-      protected String toString(byte[] ranges, int dimension) {
-        return InetAddressRangeField.toString(ranges, dimension);
-      }
-    };
-  }
-
-  /**
-   * Returns the String representation for the range at the given dimension
-   * @param ranges the encoded ranges, never null
-   * @param dimension the dimension of interest (not used for this field)
-   * @return The string representation for the range at the provided dimension
-   */
-  private static String toString(byte[] ranges, int dimension) {
-    byte[] min = new byte[BYTES];
-    System.arraycopy(ranges, 0, min, 0, BYTES);
-    byte[] max = new byte[BYTES];
-    System.arraycopy(ranges, BYTES, max, 0, BYTES);
-    return "[" + InetAddressPoint.decode(min) + " : " + InetAddressPoint.decode(max) + "]";
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/sandbox/src/java/org/apache/lucene/document/IntRangeField.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/IntRangeField.java b/lucene/sandbox/src/java/org/apache/lucene/document/IntRangeField.java
deleted file mode 100644
index 53a3311..0000000
--- a/lucene/sandbox/src/java/org/apache/lucene/document/IntRangeField.java
+++ /dev/null
@@ -1,282 +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.lucene.document;
-
-import org.apache.lucene.document.RangeFieldQuery.QueryType;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.NumericUtils;
-
-/**
- * An indexed Integer Range field.
- * <p>
- * This field indexes dimensional ranges defined as min/max pairs. It supports
- * up to a maximum of 4 dimensions (indexed as 8 numeric values). With 1 dimension representing a single integer range,
- * 2 dimensions representing a bounding box, 3 dimensions a bounding cube, and 4 dimensions a tesseract.
- * <p>
- * Multiple values for the same field in one document is supported, and open ended ranges can be defined using
- * {@code Integer.MIN_VALUE} and {@code Integer.MAX_VALUE}.
- *
- * <p>
- * This field defines the following static factory methods for common search operations over integer ranges:
- * <ul>
- *   <li>{@link #newIntersectsQuery newIntersectsQuery()} matches ranges that intersect the defined search range.
- *   <li>{@link #newWithinQuery newWithinQuery()} matches ranges that are within the defined search range.
- *   <li>{@link #newContainsQuery newContainsQuery()} matches ranges that contain the defined search range.
- * </ul>
- */
-public class IntRangeField extends Field {
-  /** stores integer values so number of bytes is 4 */
-  public static final int BYTES = Integer.BYTES;
-
-  /**
-   * Create a new IntRangeField type, from min/max parallel arrays
-   *
-   * @param name field name. must not be null.
-   * @param min range min values; each entry is the min value for the dimension
-   * @param max range max values; each entry is the max value for the dimension
-   */
-  public IntRangeField(String name, final int[] min, final int[] max) {
-    super(name, getType(min.length));
-    setRangeValues(min, max);
-  }
-
-  /** set the field type */
-  private static FieldType getType(int dimensions) {
-    if (dimensions > 4) {
-      throw new IllegalArgumentException("IntRangeField does not support greater than 4 dimensions");
-    }
-
-    FieldType ft = new FieldType();
-    // dimensions is set as 2*dimension size (min/max per dimension)
-    ft.setDimensions(dimensions*2, BYTES);
-    ft.freeze();
-    return ft;
-  }
-
-  /**
-   * Changes the values of the field.
-   * @param min array of min values. (accepts {@code Integer.NEGATIVE_INFINITY})
-   * @param max array of max values. (accepts {@code Integer.POSITIVE_INFINITY})
-   * @throws IllegalArgumentException if {@code min} or {@code max} is invalid
-   */
-  public void setRangeValues(int[] min, int[] max) {
-    checkArgs(min, max);
-    if (min.length*2 != type.pointDimensionCount() || max.length*2 != type.pointDimensionCount()) {
-      throw new IllegalArgumentException("field (name=" + name + ") uses " + type.pointDimensionCount()/2
-          + " dimensions; cannot change to (incoming) " + min.length + " dimensions");
-    }
-
-    final byte[] bytes;
-    if (fieldsData == null) {
-      bytes = new byte[BYTES*2*min.length];
-      fieldsData = new BytesRef(bytes);
-    } else {
-      bytes = ((BytesRef)fieldsData).bytes;
-    }
-    verifyAndEncode(min, max, bytes);
-  }
-
-  /** validate the arguments */
-  private static void checkArgs(final int[] min, final int[] max) {
-    if (min == null || max == null || min.length == 0 || max.length == 0) {
-      throw new IllegalArgumentException("min/max range values cannot be null or empty");
-    }
-    if (min.length != max.length) {
-      throw new IllegalArgumentException("min/max ranges must agree");
-    }
-    if (min.length > 4) {
-      throw new IllegalArgumentException("IntRangeField does not support greater than 4 dimensions");
-    }
-  }
-
-  /**
-   * Encodes the min, max ranges into a byte array
-   */
-  private static byte[] encode(int[] min, int[] max) {
-    checkArgs(min, max);
-    byte[] b = new byte[BYTES*2*min.length];
-    verifyAndEncode(min, max, b);
-    return b;
-  }
-
-  /**
-   * encode the ranges into a sortable byte array ({@code Double.NaN} not allowed)
-   * <p>
-   * example for 4 dimensions (8 bytes per dimension value):
-   * minD1 ... minD4 | maxD1 ... maxD4
-   */
-  static void verifyAndEncode(int[] min, int[] max, byte[] bytes) {
-    for (int d=0,i=0,j=min.length*BYTES; d<min.length; ++d, i+=BYTES, j+=BYTES) {
-      if (Double.isNaN(min[d])) {
-        throw new IllegalArgumentException("invalid min value (" + Double.NaN + ")" + " in IntRangeField");
-      }
-      if (Double.isNaN(max[d])) {
-        throw new IllegalArgumentException("invalid max value (" + Double.NaN + ")" + " in IntRangeField");
-      }
-      if (min[d] > max[d]) {
-        throw new IllegalArgumentException("min value (" + min[d] + ") is greater than max value (" + max[d] + ")");
-      }
-      encode(min[d], bytes, i);
-      encode(max[d], bytes, j);
-    }
-  }
-
-  /** encode the given value into the byte array at the defined offset */
-  private static void encode(int val, byte[] bytes, int offset) {
-    NumericUtils.intToSortableBytes(val, bytes, offset);
-  }
-
-  /**
-   * Get the min value for the given dimension
-   * @param dimension the dimension, always positive
-   * @return the decoded min value
-   */
-  public int getMin(int dimension) {
-    if (dimension < 0 || dimension >= type.pointDimensionCount()/2) {
-      throw new IllegalArgumentException("dimension request (" + dimension +
-          ") out of bounds for field (name=" + name + " dimensions=" + type.pointDimensionCount()/2 + "). ");
-    }
-    return decodeMin(((BytesRef)fieldsData).bytes, dimension);
-  }
-
-  /**
-   * Get the max value for the given dimension
-   * @param dimension the dimension, always positive
-   * @return the decoded max value
-   */
-  public int getMax(int dimension) {
-    if (dimension < 0 || dimension >= type.pointDimensionCount()/2) {
-      throw new IllegalArgumentException("dimension request (" + dimension +
-          ") out of bounds for field (name=" + name + " dimensions=" + type.pointDimensionCount()/2 + "). ");
-    }
-    return decodeMax(((BytesRef)fieldsData).bytes, dimension);
-  }
-
-  /** decodes the min value (for the defined dimension) from the encoded input byte array */
-  static int decodeMin(byte[] b, int dimension) {
-    int offset = dimension*BYTES;
-    return NumericUtils.sortableBytesToInt(b, offset);
-  }
-
-  /** decodes the max value (for the defined dimension) from the encoded input byte array */
-  static int decodeMax(byte[] b, int dimension) {
-    int offset = b.length/2 + dimension*BYTES;
-    return NumericUtils.sortableBytesToInt(b, offset);
-  }
-
-  /**
-   * Create a query for matching indexed ranges that intersect the defined range.
-   * @param field field name. must not be null.
-   * @param min array of min values. (accepts {@code Integer.MIN_VALUE})
-   * @param max array of max values. (accepts {@code Integer.MAX_VALUE})
-   * @return query for matching intersecting ranges (overlap, within, or contains)
-   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
-   */
-  public static Query newIntersectsQuery(String field, final int[] min, final int[] max) {
-    return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.INTERSECTS) {
-      @Override
-      protected String toString(byte[] ranges, int dimension) {
-        return IntRangeField.toString(ranges, dimension);
-      }
-    };
-  }
-
-  /**
-   * Create a query for matching indexed ranges that contain the defined range.
-   * @param field field name. must not be null.
-   * @param min array of min values. (accepts {@code Integer.MIN_VALUE})
-   * @param max array of max values. (accepts {@code Integer.MAX_VALUE})
-   * @return query for matching ranges that contain the defined range
-   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
-   */
-  public static Query newContainsQuery(String field, final int[] min, final int[] max) {
-    return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.CONTAINS) {
-      @Override
-      protected String toString(byte[] ranges, int dimension) {
-        return IntRangeField.toString(ranges, dimension);
-      }
-    };
-  }
-
-  /**
-   * Create a query for matching indexed ranges that are within the defined range.
-   * @param field field name. must not be null.
-   * @param min array of min values. (accepts {@code Integer.MIN_VALUE})
-   * @param max array of max values. (accepts {@code Integer.MAX_VALUE})
-   * @return query for matching ranges within the defined range
-   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
-   */
-  public static Query newWithinQuery(String field, final int[] min, final int[] max) {
-    checkArgs(min, max);
-    return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.WITHIN) {
-      @Override
-      protected String toString(byte[] ranges, int dimension) {
-        return IntRangeField.toString(ranges, dimension);
-      }
-    };
-  }
-
-  /**
-   * Create a query for matching indexed ranges that cross the defined range.
-   * A CROSSES is defined as any set of ranges that are not disjoint and not wholly contained by
-   * the query. Effectively, its the complement of union(WITHIN, DISJOINT).
-   * @param field field name. must not be null.
-   * @param min array of min values. (accepts {@code Integer.MIN_VALUE})
-   * @param max array of max values. (accepts {@code Integer.MAX_VALUE})
-   * @return query for matching ranges within the defined range
-   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
-   */
-  public static Query newCrossesQuery(String field, final int[] min, final int[] max) {
-    checkArgs(min, max);
-    return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.CROSSES) {
-      @Override
-      protected String toString(byte[] ranges, int dimension) {
-        return IntRangeField.toString(ranges, dimension);
-      }
-    };
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder();
-    sb.append(getClass().getSimpleName());
-    sb.append(" <");
-    sb.append(name);
-    sb.append(':');
-    byte[] b = ((BytesRef)fieldsData).bytes;
-    toString(b, 0);
-    for (int d=1; d<type.pointDimensionCount(); ++d) {
-      sb.append(' ');
-      toString(b, d);
-    }
-    sb.append('>');
-
-    return sb.toString();
-  }
-
-  /**
-   * Returns the String representation for the range at the given dimension
-   * @param ranges the encoded ranges, never null
-   * @param dimension the dimension of interest
-   * @return The string representation for the range at the provided dimension
-   */
-  private static String toString(byte[] ranges, int dimension) {
-    return "[" + Integer.toString(decodeMin(ranges, dimension)) + " : "
-        + Integer.toString(decodeMax(ranges, dimension)) + "]";
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/sandbox/src/java/org/apache/lucene/document/LongRangeField.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LongRangeField.java b/lucene/sandbox/src/java/org/apache/lucene/document/LongRangeField.java
deleted file mode 100644
index 7addaf5..0000000
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LongRangeField.java
+++ /dev/null
@@ -1,280 +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.lucene.document;
-
-import org.apache.lucene.document.RangeFieldQuery.QueryType;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.NumericUtils;
-
-/**
- * An indexed Long Range field.
- * <p>
- * This field indexes dimensional ranges defined as min/max pairs. It supports
- * up to a maximum of 4 dimensions (indexed as 8 numeric values). With 1 dimension representing a single long range,
- * 2 dimensions representing a bounding box, 3 dimensions a bounding cube, and 4 dimensions a tesseract.
- * <p>
- * Multiple values for the same field in one document is supported, and open ended ranges can be defined using
- * {@code Long.MIN_VALUE} and {@code Long.MAX_VALUE}.
- *
- * <p>
- * This field defines the following static factory methods for common search operations over long ranges:
- * <ul>
- *   <li>{@link #newIntersectsQuery newIntersectsQuery()} matches ranges that intersect the defined search range.
- *   <li>{@link #newWithinQuery newWithinQuery()} matches ranges that are within the defined search range.
- *   <li>{@link #newContainsQuery newContainsQuery()} matches ranges that contain the defined search range.
- * </ul>
- */
-public class LongRangeField extends Field {
-  /** stores long values so number of bytes is 8 */
-  public static final int BYTES = Long.BYTES;
-
-  /**
-   * Create a new LongRangeField type, from min/max parallel arrays
-   *
-   * @param name field name. must not be null.
-   * @param min range min values; each entry is the min value for the dimension
-   * @param max range max values; each entry is the max value for the dimension
-   */
-  public LongRangeField(String name, final long[] min, final long[] max) {
-    super(name, getType(min.length));
-    setRangeValues(min, max);
-  }
-
-  /** set the field type */
-  private static FieldType getType(int dimensions) {
-    if (dimensions > 4) {
-      throw new IllegalArgumentException("LongRangeField does not support greater than 4 dimensions");
-    }
-
-    FieldType ft = new FieldType();
-    // dimensions is set as 2*dimension size (min/max per dimension)
-    ft.setDimensions(dimensions*2, BYTES);
-    ft.freeze();
-    return ft;
-  }
-
-  /**
-   * Changes the values of the field.
-   * @param min array of min values. (accepts {@code Long.MIN_VALUE})
-   * @param max array of max values. (accepts {@code Long.MAX_VALUE})
-   * @throws IllegalArgumentException if {@code min} or {@code max} is invalid
-   */
-  public void setRangeValues(long[] min, long[] max) {
-    checkArgs(min, max);
-    if (min.length*2 != type.pointDimensionCount() || max.length*2 != type.pointDimensionCount()) {
-      throw new IllegalArgumentException("field (name=" + name + ") uses " + type.pointDimensionCount()/2
-          + " dimensions; cannot change to (incoming) " + min.length + " dimensions");
-    }
-
-    final byte[] bytes;
-    if (fieldsData == null) {
-      bytes = new byte[BYTES*2*min.length];
-      fieldsData = new BytesRef(bytes);
-    } else {
-      bytes = ((BytesRef)fieldsData).bytes;
-    }
-    verifyAndEncode(min, max, bytes);
-  }
-
-  /** validate the arguments */
-  private static void checkArgs(final long[] min, final long[] max) {
-    if (min == null || max == null || min.length == 0 || max.length == 0) {
-      throw new IllegalArgumentException("min/max range values cannot be null or empty");
-    }
-    if (min.length != max.length) {
-      throw new IllegalArgumentException("min/max ranges must agree");
-    }
-    if (min.length > 4) {
-      throw new IllegalArgumentException("LongRangeField does not support greater than 4 dimensions");
-    }
-  }
-
-  /** Encodes the min, max ranges into a byte array */
-  private static byte[] encode(long[] min, long[] max) {
-    checkArgs(min, max);
-    byte[] b = new byte[BYTES*2*min.length];
-    verifyAndEncode(min, max, b);
-    return b;
-  }
-
-  /**
-   * encode the ranges into a sortable byte array ({@code Double.NaN} not allowed)
-   * <p>
-   * example for 4 dimensions (8 bytes per dimension value):
-   * minD1 ... minD4 | maxD1 ... maxD4
-   */
-  static void verifyAndEncode(long[] min, long[] max, byte[] bytes) {
-    for (int d=0,i=0,j=min.length*BYTES; d<min.length; ++d, i+=BYTES, j+=BYTES) {
-      if (Double.isNaN(min[d])) {
-        throw new IllegalArgumentException("invalid min value (" + Double.NaN + ")" + " in IntRangeField");
-      }
-      if (Double.isNaN(max[d])) {
-        throw new IllegalArgumentException("invalid max value (" + Double.NaN + ")" + " in IntRangeField");
-      }
-      if (min[d] > max[d]) {
-        throw new IllegalArgumentException("min value (" + min[d] + ") is greater than max value (" + max[d] + ")");
-      }
-      encode(min[d], bytes, i);
-      encode(max[d], bytes, j);
-    }
-  }
-
-  /** encode the given value into the byte array at the defined offset */
-  private static void encode(long val, byte[] bytes, int offset) {
-    NumericUtils.longToSortableBytes(val, bytes, offset);
-  }
-
-  /**
-   * Get the min value for the given dimension
-   * @param dimension the dimension, always positive
-   * @return the decoded min value
-   */
-  public long getMin(int dimension) {
-    if (dimension < 0 || dimension >= type.pointDimensionCount()/2) {
-      throw new IllegalArgumentException("dimension request (" + dimension +
-          ") out of bounds for field (name=" + name + " dimensions=" + type.pointDimensionCount()/2 + "). ");
-    }
-    return decodeMin(((BytesRef)fieldsData).bytes, dimension);
-  }
-
-  /**
-   * Get the max value for the given dimension
-   * @param dimension the dimension, always positive
-   * @return the decoded max value
-   */
-  public long getMax(int dimension) {
-    if (dimension < 0 || dimension >= type.pointDimensionCount()/2) {
-      throw new IllegalArgumentException("dimension request (" + dimension +
-          ") out of bounds for field (name=" + name + " dimensions=" + type.pointDimensionCount()/2 + "). ");
-    }
-    return decodeMax(((BytesRef)fieldsData).bytes, dimension);
-  }
-
-  /** decodes the min value (for the defined dimension) from the encoded input byte array */
-  static long decodeMin(byte[] b, int dimension) {
-    int offset = dimension*BYTES;
-    return NumericUtils.sortableBytesToLong(b, offset);
-  }
-
-  /** decodes the max value (for the defined dimension) from the encoded input byte array */
-  static long decodeMax(byte[] b, int dimension) {
-    int offset = b.length/2 + dimension*BYTES;
-    return NumericUtils.sortableBytesToLong(b, offset);
-  }
-
-  /**
-   * Create a query for matching indexed ranges that intersect the defined range.
-   * @param field field name. must not be null.
-   * @param min array of min values. (accepts {@code Long.MIN_VALUE})
-   * @param max array of max values. (accepts {@code Long.MAX_VALUE})
-   * @return query for matching intersecting ranges (overlap, within, or contains)
-   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
-   */
-  public static Query newIntersectsQuery(String field, final long[] min, final long[] max) {
-    return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.INTERSECTS) {
-      @Override
-      protected String toString(byte[] ranges, int dimension) {
-        return LongRangeField.toString(ranges, dimension);
-      }
-    };
-  }
-
-  /**
-   * Create a query for matching indexed ranges that contain the defined range.
-   * @param field field name. must not be null.
-   * @param min array of min values. (accepts {@code Long.MIN_VALUE})
-   * @param max array of max values. (accepts {@code Long.MAX_VALUE})
-   * @return query for matching ranges that contain the defined range
-   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
-   */
-  public static Query newContainsQuery(String field, final long[] min, final long[] max) {
-    return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.CONTAINS) {
-      @Override
-      protected String toString(byte[] ranges, int dimension) {
-        return LongRangeField.toString(ranges, dimension);
-      }
-    };
-  }
-
-  /**
-   * Create a query for matching indexed ranges that are within the defined range.
-   * @param field field name. must not be null.
-   * @param min array of min values. (accepts {@code Long.MIN_VALUE})
-   * @param max array of max values. (accepts {@code Long.MAX_VALUE})
-   * @return query for matching ranges within the defined range
-   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
-   */
-  public static Query newWithinQuery(String field, final long[] min, final long[] max) {
-    checkArgs(min, max);
-    return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.WITHIN) {
-      @Override
-      protected String toString(byte[] ranges, int dimension) {
-        return LongRangeField.toString(ranges, dimension);
-      }
-    };
-  }
-
-  /**
-   * Create a query for matching indexed ranges that cross the defined range.
-   * A CROSSES is defined as any set of ranges that are not disjoint and not wholly contained by
-   * the query. Effectively, its the complement of union(WITHIN, DISJOINT).
-   * @param field field name. must not be null.
-   * @param min array of min values. (accepts {@code Long.MIN_VALUE})
-   * @param max array of max values. (accepts {@code Long.MAX_VALUE})
-   * @return query for matching ranges within the defined range
-   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
-   */
-  public static Query newCrossesQuery(String field, final long[] min, final long[] max) {
-    checkArgs(min, max);
-    return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.CROSSES) {
-      @Override
-      protected String toString(byte[] ranges, int dimension) {
-        return LongRangeField.toString(ranges, dimension);
-      }
-    };
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder();
-    sb.append(getClass().getSimpleName());
-    sb.append(" <");
-    sb.append(name);
-    sb.append(':');
-    byte[] b = ((BytesRef)fieldsData).bytes;
-    toString(b, 0);
-    for (int d=1; d<type.pointDimensionCount(); ++d) {
-      sb.append(' ');
-      toString(b, d);
-    }
-    sb.append('>');
-
-    return sb.toString();
-  }
-
-  /**
-   * Returns the String representation for the range at the given dimension
-   * @param ranges the encoded ranges, never null
-   * @param dimension the dimension of interest
-   * @return The string representation for the range at the provided dimension
-   */
-  private static String toString(byte[] ranges, int dimension) {
-    return "[" + Long.toString(decodeMin(ranges, dimension)) + " : "
-        + Long.toString(decodeMax(ranges, dimension)) + "]";
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/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
deleted file mode 100644
index 10f10fa..0000000
--- a/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java
+++ /dev/null
@@ -1,340 +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.lucene.document;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Objects;
-import java.util.function.IntPredicate;
-import java.util.function.Predicate;
-
-import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.PointValues;
-import org.apache.lucene.index.PointValues.Relation;
-import org.apache.lucene.index.PointValues.IntersectVisitor;
-import org.apache.lucene.search.ConstantScoreScorer;
-import org.apache.lucene.search.ConstantScoreWeight;
-import org.apache.lucene.search.DocIdSet;
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.Weight;
-import org.apache.lucene.util.DocIdSetBuilder;
-import org.apache.lucene.util.StringHelper;
-
-/**
- * Query class for searching {@code RangeField} types by a defined {@link Relation}.
- */
-abstract class RangeFieldQuery extends Query {
-  /** field name */
-  final String field;
-  /** query relation
-   * intersects: {@code CELL_CROSSES_QUERY},
-   * contains: {@code CELL_CONTAINS_QUERY},
-   * within: {@code CELL_WITHIN_QUERY} */
-  final QueryType queryType;
-  /** number of dimensions - max 4 */
-  final int numDims;
-  /** ranges encoded as a sortable byte array */
-  final byte[] ranges;
-  /** number of bytes per dimension */
-  final int bytesPerDim;
-
-  /** Used by {@code RangeFieldQuery} to check how each internal or leaf node relates to the query. */
-  enum QueryType {
-    /** Use this for intersects queries. */
-    INTERSECTS,
-    /** Use this for within queries. */
-    WITHIN,
-    /** Use this for contains */
-    CONTAINS,
-    /** Use this for crosses queries */
-    CROSSES
-  }
-
-  /**
-   * Create a query for searching indexed ranges that match the provided relation.
-   * @param field field name. must not be null.
-   * @param ranges encoded range values; this is done by the {@code RangeField} implementation
-   * @param queryType the query relation
-   */
-  RangeFieldQuery(String field, final byte[] ranges, final int numDims, final QueryType queryType) {
-    checkArgs(field, ranges, numDims);
-    if (queryType == null) {
-      throw new IllegalArgumentException("Query type cannot be null");
-    }
-    this.field = field;
-    this.queryType = queryType;
-    this.numDims = numDims;
-    this.ranges = ranges;
-    this.bytesPerDim = ranges.length / (2*numDims);
-  }
-
-  /** check input arguments */
-  private static void checkArgs(String field, final byte[] ranges, final int numDims) {
-    if (field == null) {
-      throw new IllegalArgumentException("field must not be null");
-    }
-    if (numDims > 4) {
-      throw new IllegalArgumentException("dimension size cannot be greater than 4");
-    }
-    if (ranges == null || ranges.length == 0) {
-      throw new IllegalArgumentException("encoded ranges cannot be null or empty");
-    }
-  }
-
-  /** Check indexed field info against the provided query data. */
-  private void checkFieldInfo(FieldInfo fieldInfo) {
-    if (fieldInfo.getPointDimensionCount()/2 != numDims) {
-      throw new IllegalArgumentException("field=\"" + field + "\" was indexed with numDims="
-          + fieldInfo.getPointDimensionCount()/2 + " but this query has numDims=" + numDims);
-    }
-  }
-
-  @Override
-  public final Weight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException {
-    return new ConstantScoreWeight(this, boost) {
-      final RangeFieldComparator target = new RangeFieldComparator();
-      private DocIdSet buildMatchingDocIdSet(LeafReader reader, PointValues values) throws IOException {
-        DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc(), values, field);
-        values.intersect(
-            new IntersectVisitor() {
-              DocIdSetBuilder.BulkAdder adder;
-              @Override
-              public void grow(int count) {
-                adder = result.grow(count);
-              }
-              @Override
-              public void visit(int docID) throws IOException {
-                adder.add(docID);
-              }
-              @Override
-              public void visit(int docID, byte[] leaf) throws IOException {
-                if (target.matches(leaf)) {
-                  adder.add(docID);
-                }
-              }
-              @Override
-              public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
-                byte[] node = getInternalRange(minPackedValue, maxPackedValue);
-                // compute range relation for BKD traversal
-                if (target.intersects(node) == false) {
-                  return Relation.CELL_OUTSIDE_QUERY;
-                } else if (target.within(node)) {
-                  // target within cell; continue traversing:
-                  return Relation.CELL_CROSSES_QUERY;
-                } else if (target.contains(node)) {
-                  // target contains cell; add iff queryType is not a CONTAINS or CROSSES query:
-                  return (queryType == QueryType.CONTAINS || queryType == QueryType.CROSSES) ?
-                      Relation.CELL_OUTSIDE_QUERY : Relation.CELL_INSIDE_QUERY;
-                }
-                // target intersects cell; continue traversing:
-                return Relation.CELL_CROSSES_QUERY;
-              }
-            });
-        return result.build();
-      }
-
-      @Override
-      public Scorer scorer(LeafReaderContext context) throws IOException {
-        LeafReader reader = context.reader();
-        PointValues values = reader.getPointValues(field);
-        if (values == null) {
-          // no docs in this segment indexed any ranges
-          return null;
-        }
-        FieldInfo fieldInfo = reader.getFieldInfos().fieldInfo(field);
-        if (fieldInfo == null) {
-          // no docs in this segment indexed this field
-          return null;
-        }
-        checkFieldInfo(fieldInfo);
-        boolean allDocsMatch = true;
-        if (values.getDocCount() == reader.maxDoc()) {
-          // if query crosses, docs need to be further scrutinized
-          byte[] range = getInternalRange(values.getMinPackedValue(), values.getMaxPackedValue());
-          // if the internal node is not equal and not contained by the query, all docs do not match
-          if (queryType == QueryType.CROSSES || (!Arrays.equals(ranges, range)
-              && (target.contains(range) == false || queryType != QueryType.WITHIN))) {
-            allDocsMatch = false;
-          }
-        } else {
-          allDocsMatch = false;
-        }
-
-        DocIdSetIterator iterator = allDocsMatch == true ?
-            DocIdSetIterator.all(reader.maxDoc()) : buildMatchingDocIdSet(reader, values).iterator();
-        return new ConstantScoreScorer(this, score(), iterator);
-      }
-
-      /** get an encoded byte representation of the internal node; this is
-       *  the lower half of the min array and the upper half of the max array */
-      private byte[] getInternalRange(byte[] min, byte[] max) {
-        byte[] range = new byte[min.length];
-        final int dimSize = numDims * bytesPerDim;
-        System.arraycopy(min, 0, range, 0, dimSize);
-        System.arraycopy(max, dimSize, range, dimSize, dimSize);
-        return range;
-      }
-    };
-  }
-
-  /**
-   * RangeFieldComparator class provides the core comparison logic for accepting or rejecting indexed
-   * {@code RangeField} types based on the defined query range and relation.
-   */
-  class RangeFieldComparator {
-    final Predicate<byte[]> predicate;
-
-    /** constructs the comparator based on the query type */
-    RangeFieldComparator() {
-      switch (queryType) {
-        case INTERSECTS:
-          predicate = this::intersects;
-          break;
-        case WITHIN:
-          predicate = this::contains;
-          break;
-        case CONTAINS:
-          predicate = this::within;
-          break;
-        case CROSSES:
-          // crosses first checks intersection (disjoint automatic fails),
-          // then ensures the query doesn't wholly contain the leaf:
-          predicate = (byte[] leaf) -> this.intersects(leaf)
-              && this.contains(leaf) == false;
-          break;
-        default:
-          throw new IllegalArgumentException("invalid queryType [" + queryType + "] found.");
-      }
-    }
-
-    /** determines if the candidate range matches the query request */
-    private boolean matches(final byte[] candidate) {
-      return (Arrays.equals(ranges, candidate) && queryType != QueryType.CROSSES)
-          || predicate.test(candidate);
-    }
-
-    /** check if query intersects candidate range */
-    private boolean intersects(final byte[] candidate) {
-      return relate((int d) -> compareMinMax(candidate, d) > 0 || compareMaxMin(candidate, d) < 0);
-    }
-
-    /** check if query is within candidate range */
-    private boolean within(final byte[] candidate) {
-      return relate((int d) -> compareMinMin(candidate, d) < 0 || compareMaxMax(candidate, d) > 0);
-    }
-
-    /** check if query contains candidate range */
-    private boolean contains(final byte[] candidate) {
-      return relate((int d) -> compareMinMin(candidate, d) > 0 || compareMaxMax(candidate, d) < 0);
-    }
-
-    /** internal method used by each relation method to test range relation logic */
-    private boolean relate(IntPredicate predicate) {
-      for (int d=0; d<numDims; ++d) {
-        if (predicate.test(d)) {
-          return false;
-        }
-      }
-      return true;
-    }
-
-    /** compare the encoded min value (for the defined query dimension) with the encoded min value in the byte array */
-    private int compareMinMin(byte[] b, int dimension) {
-      // convert dimension to offset:
-      dimension *= bytesPerDim;
-      return StringHelper.compare(bytesPerDim, ranges, dimension, b, dimension);
-    }
-
-    /** compare the encoded min value (for the defined query dimension) with the encoded max value in the byte array */
-    private int compareMinMax(byte[] b, int dimension) {
-      // convert dimension to offset:
-      dimension *= bytesPerDim;
-      return StringHelper.compare(bytesPerDim, ranges, dimension, b, numDims * bytesPerDim + dimension);
-    }
-
-    /** compare the encoded max value (for the defined query dimension) with the encoded min value in the byte array */
-    private int compareMaxMin(byte[] b, int dimension) {
-      // convert dimension to offset:
-      dimension *= bytesPerDim;
-      return StringHelper.compare(bytesPerDim, ranges, numDims * bytesPerDim + dimension, b, dimension);
-    }
-
-    /** compare the encoded max value (for the defined query dimension) with the encoded max value in the byte array */
-    private int compareMaxMax(byte[] b, int dimension) {
-      // convert dimension to max offset:
-      dimension = numDims * bytesPerDim + dimension * bytesPerDim;
-      return StringHelper.compare(bytesPerDim, ranges, dimension, b, dimension);
-    }
-  }
-
-  @Override
-  public int hashCode() {
-    int hash = classHash();
-    hash = 31 * hash + field.hashCode();
-    hash = 31 * hash + numDims;
-    hash = 31 * hash + queryType.hashCode();
-    hash = 31 * hash + Arrays.hashCode(ranges);
-
-    return hash;
-  }
-
-  @Override
-  public final boolean equals(Object o) {
-    return sameClassAs(o) &&
-        equalsTo(getClass().cast(o));
-  }
-
-  protected boolean equalsTo(RangeFieldQuery other) {
-    return Objects.equals(field, other.field) &&
-        numDims == other.numDims &&
-        Arrays.equals(ranges, other.ranges) &&
-        other.queryType == queryType;
-  }
-
-  @Override
-  public String toString(String field) {
-    StringBuilder sb = new StringBuilder();
-    if (this.field.equals(field) == false) {
-      sb.append(this.field);
-      sb.append(':');
-    }
-    sb.append("<ranges:");
-    sb.append(toString(ranges, 0));
-    for (int d=1; d<numDims; ++d) {
-      sb.append(' ');
-      sb.append(toString(ranges, d));
-    }
-    sb.append('>');
-
-    return sb.toString();
-  }
-
-  /**
-   * Returns a string of a single value in a human-readable format for debugging.
-   * This is used by {@link #toString()}.
-   *
-   * @param dimension dimension of the particular value
-   * @param ranges encoded ranges, never null
-   * @return human readable value for debugging
-   */
-  protected abstract String toString(byte[] ranges, int dimension);
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/sandbox/src/java/org/apache/lucene/document/package.html
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/package.html b/lucene/sandbox/src/java/org/apache/lucene/document/package.html
index b6a077e..4783b5e 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/package.html
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/package.html
@@ -26,8 +26,7 @@
 This package contains several point types:
 <ul>
    <li>{@link org.apache.lucene.document.BigIntegerPoint BigIntegerPoint} for 128-bit integers</li>
-   <li>{@link org.apache.lucene.document.InetAddressPoint InetAddressPoint} for IPv4 and IPv6 network addresses</li>
-   <li>{@link org.apache.lucene.document.LatLonPoint LatLonPoint} for latitude/longitude geospatial points</li> 
+   <li>{@link org.apache.lucene.document.LatLonPoint LatLonPoint} for latitude/longitude geospatial points</li>
 </ul>
 </body>
 </html>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/sandbox/src/test/org/apache/lucene/document/TestDoubleRangeField.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestDoubleRangeField.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestDoubleRangeField.java
index 188aab6..8a8130b 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestDoubleRangeField.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestDoubleRangeField.java
@@ -30,11 +30,11 @@ public class TestDoubleRangeField extends LuceneTestCase {
     IllegalArgumentException expected;
 
     expected = expectThrows(IllegalArgumentException.class, () ->
-        doc.add(new DoubleRangeField(FIELD_NAME, new double[] {Double.NaN}, new double[] {5})));
+        doc.add(new DoubleRange(FIELD_NAME, new double[] {Double.NaN}, new double[] {5})));
     assertTrue(expected.getMessage().contains("invalid min value"));
 
     expected = expectThrows(IllegalArgumentException.class, () ->
-        doc.add(new DoubleRangeField(FIELD_NAME, new double[] {5}, new double[] {Double.NaN})));
+        doc.add(new DoubleRange(FIELD_NAME, new double[] {5}, new double[] {Double.NaN})));
     assertTrue(expected.getMessage().contains("invalid max value"));
   }
 
@@ -43,7 +43,7 @@ public class TestDoubleRangeField extends LuceneTestCase {
     Document doc = new Document();
     IllegalArgumentException expected;
     expected = expectThrows(IllegalArgumentException.class, () ->
-        doc.add(new DoubleRangeField(FIELD_NAME, new double[] {5, 6}, new double[] {5})));
+        doc.add(new DoubleRange(FIELD_NAME, new double[] {5, 6}, new double[] {5})));
     assertTrue(expected.getMessage().contains("min/max ranges must agree"));
   }
 
@@ -52,7 +52,7 @@ public class TestDoubleRangeField extends LuceneTestCase {
     Document doc = new Document();
     IllegalArgumentException expected;
     expected = expectThrows(IllegalArgumentException.class, () ->
-        doc.add(new DoubleRangeField(FIELD_NAME, new double[] {1, 2, 3, 4, 5}, new double[] {5})));
+        doc.add(new DoubleRange(FIELD_NAME, new double[] {1, 2, 3, 4, 5}, new double[] {5})));
     assertTrue(expected.getMessage().contains("does not support greater than 4 dimensions"));
   }
 
@@ -61,7 +61,7 @@ public class TestDoubleRangeField extends LuceneTestCase {
     Document doc = new Document();
     IllegalArgumentException expected;
     expected = expectThrows(IllegalArgumentException.class, () ->
-      doc.add(new DoubleRangeField(FIELD_NAME, new double[] {3, 4}, new double[] {1, 2})));
+      doc.add(new DoubleRange(FIELD_NAME, new double[] {3, 4}, new double[] {1, 2})));
     assertTrue(expected.getMessage().contains("is greater than max value"));
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
deleted file mode 100644
index 0e0901b..0000000
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
+++ /dev/null
@@ -1,176 +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.lucene.document;
-
-import java.net.InetAddress;
-
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.LuceneTestCase;
-
-/** Simple tests for {@link InetAddressPoint} */
-public class TestInetAddressPoint extends LuceneTestCase {
-
-  /** Add a single address and search for it */
-  public void testBasics() throws Exception {
-    Directory dir = newDirectory();
-    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
-
-    // add a doc with an address
-    Document document = new Document();
-    InetAddress address = InetAddress.getByName("1.2.3.4");
-    document.add(new InetAddressPoint("field", address));
-    writer.addDocument(document);
-    
-    // search and verify we found our doc
-    IndexReader reader = writer.getReader();
-    IndexSearcher searcher = newSearcher(reader);
-    assertEquals(1, searcher.count(InetAddressPoint.newExactQuery("field", address)));
-    assertEquals(1, searcher.count(InetAddressPoint.newPrefixQuery("field", address, 24)));
-    assertEquals(1, searcher.count(InetAddressPoint.newRangeQuery("field", InetAddress.getByName("1.2.3.3"), InetAddress.getByName("1.2.3.5"))));
-    assertEquals(1, searcher.count(InetAddressPoint.newSetQuery("field", InetAddress.getByName("1.2.3.4"))));
-    assertEquals(1, searcher.count(InetAddressPoint.newSetQuery("field", InetAddress.getByName("1.2.3.4"), InetAddress.getByName("1.2.3.5"))));
-    assertEquals(0, searcher.count(InetAddressPoint.newSetQuery("field", InetAddress.getByName("1.2.3.3"))));
-    assertEquals(0, searcher.count(InetAddressPoint.newSetQuery("field")));
-
-    reader.close();
-    writer.close();
-    dir.close();
-  }
-  
-  /** Add a single address and search for it */
-  public void testBasicsV6() throws Exception {
-    Directory dir = newDirectory();
-    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
-
-    // add a doc with an address
-    Document document = new Document();
-    InetAddress address = InetAddress.getByName("fec0::f66d");
-    document.add(new InetAddressPoint("field", address));
-    writer.addDocument(document);
-    
-    // search and verify we found our doc
-    IndexReader reader = writer.getReader();
-    IndexSearcher searcher = newSearcher(reader);
-    assertEquals(1, searcher.count(InetAddressPoint.newExactQuery("field", address)));
-    assertEquals(1, searcher.count(InetAddressPoint.newPrefixQuery("field", address, 64)));
-    assertEquals(1, searcher.count(InetAddressPoint.newRangeQuery("field", InetAddress.getByName("fec0::f66c"), InetAddress.getByName("fec0::f66e"))));
-
-    reader.close();
-    writer.close();
-    dir.close();
-  }
-    
-  public void testToString() throws Exception {
-    assertEquals("InetAddressPoint <field:1.2.3.4>", new InetAddressPoint("field", InetAddress.getByName("1.2.3.4")).toString());
-    assertEquals("InetAddressPoint <field:1.2.3.4>", new InetAddressPoint("field", InetAddress.getByName("::FFFF:1.2.3.4")).toString());
-    assertEquals("InetAddressPoint <field:[fdc8:57ed:f042:ad1:f66d:4ff:fe90:ce0c]>", new InetAddressPoint("field", InetAddress.getByName("fdc8:57ed:f042:0ad1:f66d:4ff:fe90:ce0c")).toString());
-    
-    assertEquals("field:[1.2.3.4 TO 1.2.3.4]", InetAddressPoint.newExactQuery("field", InetAddress.getByName("1.2.3.4")).toString());
-    assertEquals("field:[0:0:0:0:0:0:0:1 TO 0:0:0:0:0:0:0:1]", InetAddressPoint.newExactQuery("field", InetAddress.getByName("::1")).toString());
-    
-    assertEquals("field:[1.2.3.0 TO 1.2.3.255]", InetAddressPoint.newPrefixQuery("field", InetAddress.getByName("1.2.3.4"), 24).toString());
-    assertEquals("field:[fdc8:57ed:f042:ad1:0:0:0:0 TO fdc8:57ed:f042:ad1:ffff:ffff:ffff:ffff]", InetAddressPoint.newPrefixQuery("field", InetAddress.getByName("fdc8:57ed:f042:0ad1:f66d:4ff:fe90:ce0c"), 64).toString());
-    assertEquals("field:{fdc8:57ed:f042:ad1:f66d:4ff:fe90:ce0c}", InetAddressPoint.newSetQuery("field", InetAddress.getByName("fdc8:57ed:f042:0ad1:f66d:4ff:fe90:ce0c")).toString());
-  }
-
-  public void testQueryEquals() throws Exception {
-    Query q1, q2;
-    q1 = InetAddressPoint.newRangeQuery("a", InetAddress.getByName("1.2.3.3"), InetAddress.getByName("1.2.3.5"));
-    q2 = InetAddressPoint.newRangeQuery("a", InetAddress.getByName("1.2.3.3"), InetAddress.getByName("1.2.3.5"));
-    assertEquals(q1, q2);
-    assertEquals(q1.hashCode(), q2.hashCode());
-    assertFalse(q1.equals(InetAddressPoint.newRangeQuery("a", InetAddress.getByName("1.2.3.3"), InetAddress.getByName("1.2.3.7"))));
-    assertFalse(q1.equals(InetAddressPoint.newRangeQuery("b", InetAddress.getByName("1.2.3.3"), InetAddress.getByName("1.2.3.5"))));
-
-    q1 = InetAddressPoint.newPrefixQuery("a", InetAddress.getByName("1.2.3.3"), 16);
-    q2 = InetAddressPoint.newPrefixQuery("a", InetAddress.getByName("1.2.3.3"), 16);
-    assertEquals(q1, q2);
-    assertEquals(q1.hashCode(), q2.hashCode());
-    assertFalse(q1.equals(InetAddressPoint.newPrefixQuery("a", InetAddress.getByName("1.1.3.5"), 16)));
-    assertFalse(q1.equals(InetAddressPoint.newPrefixQuery("a", InetAddress.getByName("1.2.3.5"), 24)));
-
-    q1 = InetAddressPoint.newExactQuery("a", InetAddress.getByName("1.2.3.3"));
-    q2 = InetAddressPoint.newExactQuery("a", InetAddress.getByName("1.2.3.3"));
-    assertEquals(q1, q2);
-    assertEquals(q1.hashCode(), q2.hashCode());
-    assertFalse(q1.equals(InetAddressPoint.newExactQuery("a", InetAddress.getByName("1.2.3.5"))));
-
-    q1 = InetAddressPoint.newSetQuery("a", InetAddress.getByName("1.2.3.3"), InetAddress.getByName("1.2.3.5"));
-    q2 = InetAddressPoint.newSetQuery("a", InetAddress.getByName("1.2.3.3"), InetAddress.getByName("1.2.3.5"));
-    assertEquals(q1, q2);
-    assertEquals(q1.hashCode(), q2.hashCode());
-    assertFalse(q1.equals(InetAddressPoint.newSetQuery("a", InetAddress.getByName("1.2.3.3"), InetAddress.getByName("1.2.3.7"))));
-  }
-
-  public void testPrefixQuery() throws Exception {
-    assertEquals(
-        InetAddressPoint.newRangeQuery("a", InetAddress.getByName("1.2.3.0"), InetAddress.getByName("1.2.3.255")),
-        InetAddressPoint.newPrefixQuery("a", InetAddress.getByName("1.2.3.127"), 24));
-    assertEquals(
-        InetAddressPoint.newRangeQuery("a", InetAddress.getByName("1.2.3.128"), InetAddress.getByName("1.2.3.255")),
-        InetAddressPoint.newPrefixQuery("a", InetAddress.getByName("1.2.3.213"), 25));
-    assertEquals(
-        InetAddressPoint.newRangeQuery("a", InetAddress.getByName("2001::a000:0"), InetAddress.getByName("2001::afff:ffff")),
-        InetAddressPoint.newPrefixQuery("a", InetAddress.getByName("2001::a6bd:fc80"), 100));
-  }
-
-  public void testNextUp() throws Exception {
-    assertEquals(InetAddress.getByName("::1"),
-        InetAddressPoint.nextUp(InetAddress.getByName("::")));
-
-    assertEquals(InetAddress.getByName("::1:0"),
-        InetAddressPoint.nextUp(InetAddress.getByName("::ffff")));
-
-    assertEquals(InetAddress.getByName("1.2.4.0"),
-        InetAddressPoint.nextUp(InetAddress.getByName("1.2.3.255")));
-
-    assertEquals(InetAddress.getByName("0.0.0.0"),
-        InetAddressPoint.nextUp(InetAddress.getByName("::fffe:ffff:ffff")));
-
-    assertEquals(InetAddress.getByName("::1:0:0:0"),
-        InetAddressPoint.nextUp(InetAddress.getByName("255.255.255.255")));
-
-    ArithmeticException e = expectThrows(ArithmeticException.class,
-        () -> InetAddressPoint.nextUp(InetAddress.getByName("ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff")));
-    assertEquals("Overflow: there is no greater InetAddress than ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff", e.getMessage());
-  }
-
-  public void testNextDown() throws Exception {
-    assertEquals(InetAddress.getByName("ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffe"),
-        InetAddressPoint.nextDown(InetAddress.getByName("ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff")));
-
-    assertEquals(InetAddress.getByName("::ffff"),
-        InetAddressPoint.nextDown(InetAddress.getByName("::1:0")));
-
-    assertEquals(InetAddress.getByName("1.2.3.255"),
-        InetAddressPoint.nextDown(InetAddress.getByName("1.2.4.0")));
-
-    assertEquals(InetAddress.getByName("::fffe:ffff:ffff"),
-        InetAddressPoint.nextDown(InetAddress.getByName("0.0.0.0")));
-
-    assertEquals(InetAddress.getByName("255.255.255.255"),
-        InetAddressPoint.nextDown(InetAddress.getByName("::1:0:0:0")));
-
-    ArithmeticException e = expectThrows(ArithmeticException.class,
-        () -> InetAddressPoint.nextDown(InetAddress.getByName("::")));
-    assertEquals("Underflow: there is no smaller InetAddress than 0:0:0:0:0:0:0:0", e.getMessage());
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/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
deleted file mode 100644
index 53f3b82..0000000
--- a/lucene/sandbox/src/test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java
+++ /dev/null
@@ -1,344 +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.lucene.search;
-
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.NumericDocValuesField;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.MultiDocValues;
-import org.apache.lucene.index.MultiFields;
-import org.apache.lucene.index.NumericDocValues;
-import org.apache.lucene.index.SerialMergeScheduler;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.FixedBitSet;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.LuceneTestCase;
-
-/**
- * Abstract class to do basic tests for a RangeField query. Testing rigor inspired by {@code BaseGeoPointTestCase}
- */
-public abstract class BaseRangeFieldQueryTestCase extends LuceneTestCase {
-  protected abstract Field newRangeField(Range box);
-
-  protected abstract Query newIntersectsQuery(Range box);
-
-  protected abstract Query newContainsQuery(Range box);
-
-  protected abstract Query newWithinQuery(Range box);
-
-  protected abstract Query newCrossesQuery(Range box);
-
-  protected abstract Range nextRange(int dimensions);
-
-  protected int dimension() {
-    return random().nextInt(4) + 1;
-  }
-
-  public void testRandomTiny() throws Exception {
-    // Make sure single-leaf-node case is OK:
-    doTestRandom(10, false);
-  }
-
-  public void testRandomMedium() throws Exception {
-    doTestRandom(10000, false);
-  }
-
-  @Nightly
-  public void testRandomBig() throws Exception {
-    doTestRandom(200000, false);
-  }
-
-  public void testMultiValued() throws Exception {
-    doTestRandom(10000, true);
-  }
-
-  private void doTestRandom(int count, boolean multiValued) throws Exception {
-    int numDocs = atLeast(count);
-    int dimensions = dimension();
-
-    if (VERBOSE) {
-      System.out.println("TEST: numDocs=" + numDocs);
-    }
-
-    Range[][] ranges = new Range[numDocs][];
-
-    boolean haveRealDoc = true;
-
-    nextdoc: for (int id=0; id<numDocs; ++id) {
-      int x = random().nextInt(20);
-      if (ranges[id] == null) {
-        ranges[id] = new Range[] {nextRange(dimensions)};
-      }
-      if (x == 17) {
-        // some docs don't have a box:
-        ranges[id][0].isMissing = true;
-        if (VERBOSE) {
-          System.out.println("  id=" + id + " is missing");
-        }
-        continue;
-      }
-
-      if (multiValued == true && random().nextBoolean()) {
-        // randomly add multi valued documents (up to 2 fields)
-        int n = random().nextInt(2) + 1;
-        ranges[id] = new Range[n];
-        for (int i=0; i<n; ++i) {
-          ranges[id][i] = nextRange(dimensions);
-        }
-      }
-
-      if (id > 0 && x < 9 && haveRealDoc) {
-        int oldID;
-        int i=0;
-        // don't step on missing ranges:
-        while (true) {
-          oldID = random().nextInt(id);
-          if (ranges[oldID][0].isMissing == false) {
-            break;
-          } else if (++i > id) {
-            continue nextdoc;
-          }
-        }
-
-        if (x == dimensions*2) {
-          // Fully identical box (use first box in case current is multivalued but old is not)
-          for (int d=0; d<dimensions; ++d) {
-            ranges[id][0].setMin(d, ranges[oldID][0].getMin(d));
-            ranges[id][0].setMax(d, ranges[oldID][0].getMax(d));
-          }
-          if (VERBOSE) {
-            System.out.println("  id=" + id + " box=" + ranges[id] + " (same box as doc=" + oldID + ")");
-          }
-        } else {
-          for (int m = 0, even = dimensions % 2; m < dimensions * 2; ++m) {
-            if (x == m) {
-              int d = (int)Math.floor(m/2);
-              // current could be multivalue but old may not be, so use first box
-              if (even == 0) {
-                ranges[id][0].setMin(d, ranges[oldID][0].getMin(d));
-                if (VERBOSE) {
-                  System.out.println("  id=" + id + " box=" + ranges[id] + " (same min[" + d + "] as doc=" + oldID + ")");
-                }
-              } else {
-                ranges[id][0].setMax(d, ranges[oldID][0].getMax(d));
-                if (VERBOSE) {
-                  System.out.println("  id=" + id + " box=" + ranges[id] + " (same max[" + d + "] as doc=" + oldID + ")");
-                }
-              }
-            }
-          }
-        }
-      }
-    }
-    verify(ranges);
-  }
-
-  private void verify(Range[][] ranges) throws Exception {
-    IndexWriterConfig iwc = newIndexWriterConfig();
-    // Else seeds may not reproduce:
-    iwc.setMergeScheduler(new SerialMergeScheduler());
-    // Else we can get O(N^2) merging
-    int mbd = iwc.getMaxBufferedDocs();
-    if (mbd != -1 && mbd < ranges.length/100) {
-      iwc.setMaxBufferedDocs(ranges.length/100);
-    }
-    Directory dir;
-    if (ranges.length > 50000) {
-      dir = newFSDirectory(createTempDir(getClass().getSimpleName()));
-    } else {
-      dir = newDirectory();
-    }
-
-    Set<Integer> deleted = new HashSet<>();
-    IndexWriter w = new IndexWriter(dir, iwc);
-    for (int id=0; id < ranges.length; ++id) {
-      Document doc = new Document();
-      doc.add(newStringField("id", ""+id, Field.Store.NO));
-      doc.add(new NumericDocValuesField("id", id));
-      if (ranges[id][0].isMissing == false) {
-        for (int n=0; n<ranges[id].length; ++n) {
-          doc.add(newRangeField(ranges[id][n]));
-        }
-      }
-      w.addDocument(doc);
-      if (id > 0 && random().nextInt(100) == 1) {
-        int idToDelete = random().nextInt(id);
-        w.deleteDocuments(new Term("id", ""+idToDelete));
-        deleted.add(idToDelete);
-        if (VERBOSE) {
-          System.out.println("  delete id=" + idToDelete);
-        }
-      }
-    }
-
-    if (random().nextBoolean()) {
-      w.forceMerge(1);
-    }
-    final IndexReader r = DirectoryReader.open(w);
-    w.close();
-    IndexSearcher s = newSearcher(r);
-
-    int dimensions = ranges[0][0].numDimensions();
-    int iters = atLeast(25);
-    Bits liveDocs = MultiFields.getLiveDocs(s.getIndexReader());
-    int maxDoc = s.getIndexReader().maxDoc();
-
-    for (int iter=0; iter<iters; ++iter) {
-      if (VERBOSE) {
-        System.out.println("\nTEST: iter=" + iter + " s=" + s);
-      }
-
-      // occasionally test open ended bounding ranges
-      Range queryRange = nextRange(dimensions);
-      int rv = random().nextInt(4);
-      Query query;
-      Range.QueryType queryType;
-      if (rv == 0) {
-        queryType = Range.QueryType.INTERSECTS;
-        query = newIntersectsQuery(queryRange);
-      } else if (rv == 1)  {
-        queryType = Range.QueryType.CONTAINS;
-        query = newContainsQuery(queryRange);
-      } else if (rv == 2) {
-        queryType = Range.QueryType.WITHIN;
-        query = newWithinQuery(queryRange);
-      } else {
-        queryType = Range.QueryType.CROSSES;
-        query = newCrossesQuery(queryRange);
-      }
-
-      if (VERBOSE) {
-        System.out.println("  query=" + query);
-      }
-
-      final FixedBitSet hits = new FixedBitSet(maxDoc);
-      s.search(query, new SimpleCollector() {
-        private int docBase;
-
-        @Override
-        public void collect(int doc) {
-          hits.set(docBase + doc);
-        }
-
-        @Override
-        protected void doSetNextReader(LeafReaderContext context) throws IOException {
-          docBase = context.docBase;
-        }
-
-        @Override
-        public boolean needsScores() { return false; }
-      });
-
-      NumericDocValues docIDToID = MultiDocValues.getNumericValues(r, "id");
-      for (int docID=0; docID<maxDoc; ++docID) {
-        assertEquals(docID, docIDToID.nextDoc());
-        int id = (int) docIDToID.longValue();
-        boolean expected;
-        if (liveDocs != null && liveDocs.get(docID) == false) {
-          // document is deleted
-          expected = false;
-        } else if (ranges[id][0].isMissing) {
-          expected = false;
-        } else {
-          expected = expectedResult(queryRange, ranges[id], queryType);
-        }
-
-        if (hits.get(docID) != expected) {
-          StringBuilder b = new StringBuilder();
-          b.append("FAIL (iter " + iter + "): ");
-          if (expected == true) {
-            b.append("id=" + id + (ranges[id].length > 1 ? " (MultiValue) " : " ") + "should match but did not\n");
-          } else {
-            b.append("id=" + id + " should not match but did\n");
-          }
-          b.append(" queryRange=" + queryRange + "\n");
-          b.append(" box" + ((ranges[id].length > 1) ? "es=" : "=" ) + ranges[id][0]);
-          for (int n=1; n<ranges[id].length; ++n) {
-            b.append(", ");
-            b.append(ranges[id][n]);
-          }
-          b.append("\n queryType=" + queryType + "\n");
-          b.append(" deleted?=" + (liveDocs != null && liveDocs.get(docID) == false));
-          fail("wrong hit (first of possibly more):\n\n" + b);
-        }
-      }
-    }
-    IOUtils.close(r, dir);
-  }
-
-  protected boolean expectedResult(Range queryRange, Range[] range, Range.QueryType queryType) {
-    for (int i=0; i<range.length; ++i) {
-      if (expectedBBoxQueryResult(queryRange, range[i], queryType) == true) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  protected boolean expectedBBoxQueryResult(Range queryRange, Range range, Range.QueryType queryType) {
-    if (queryRange.isEqual(range) && queryType != Range.QueryType.CROSSES) {
-      return true;
-    }
-    Range.QueryType relation = range.relate(queryRange);
-    if (queryType == Range.QueryType.INTERSECTS) {
-      return relation != null;
-    } else if (queryType == Range.QueryType.CROSSES) {
-      // by definition, RangeFields that CONTAIN the query are also considered to cross
-      return relation == queryType || relation == Range.QueryType.CONTAINS;
-    }
-    return relation == queryType;
-  }
-
-  abstract static class Range {
-    protected boolean isMissing = false;
-
-    enum QueryType { INTERSECTS, WITHIN, CONTAINS, CROSSES }
-
-    protected abstract int numDimensions();
-    protected abstract Object getMin(int dim);
-    protected abstract void setMin(int dim, Object val);
-    protected abstract Object getMax(int dim);
-    protected abstract void setMax(int dim, Object val);
-    protected abstract boolean isEqual(Range other);
-    protected abstract boolean isDisjoint(Range other);
-    protected abstract boolean isWithin(Range other);
-    protected abstract boolean contains(Range other);
-
-    protected QueryType relate(Range other) {
-      if (isDisjoint(other)) {
-        // if disjoint; return null:
-        return null;
-      } else if (isWithin(other)) {
-        return QueryType.WITHIN;
-      } else if (contains(other)) {
-        return QueryType.CONTAINS;
-      }
-      return QueryType.CROSSES;
-    }
-  }
-}


[15/43] lucene-solr:feature/autoscaling: LUCENE-7695: support synonyms in ComplexPhraseQueryParser

Posted by sh...@apache.org.
LUCENE-7695: support synonyms in ComplexPhraseQueryParser


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

Branch: refs/heads/feature/autoscaling
Commit: 8a5492930eff393de824450f77f27d98a204df3d
Parents: d844207
Author: Mikhail Khludnev <mk...@apache.org>
Authored: Sun Mar 5 12:24:47 2017 +0300
Committer: Mikhail Khludnev <mk...@apache.org>
Committed: Wed Mar 8 11:20:35 2017 +0300

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  3 ++
 .../complexPhrase/ComplexPhraseQueryParser.java | 21 +++++++++---
 .../complexPhrase/TestComplexPhraseQuery.java   | 36 +++++++++++++++++---
 3 files changed, 52 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8a549293/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index b067fde..a8f7ee4 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -204,6 +204,9 @@ Improvements
   IndexInput description instead of plain IOException (Mike Drob via
   Mike McCandless)
 
+* LUCENE-7695: ComplexPhraseQueryParser to support query time synonyms (Markus Jelsma
+  via Mikhail Khludnev) 
+
 Optimizations
 
 * LUCENE-7641: Optimized point range queries to compute documents that do not

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8a549293/lucene/queryparser/src/java/org/apache/lucene/queryparser/complexPhrase/ComplexPhraseQueryParser.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/complexPhrase/ComplexPhraseQueryParser.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/complexPhrase/ComplexPhraseQueryParser.java
index 6e18960..32f4fb3 100644
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/complexPhrase/ComplexPhraseQueryParser.java
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/complexPhrase/ComplexPhraseQueryParser.java
@@ -28,6 +28,7 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.queryparser.classic.ParseException;
 import org.apache.lucene.queryparser.classic.QueryParser;
 import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.BoostQuery;
 import org.apache.lucene.search.IndexSearcher;
@@ -35,6 +36,7 @@ import org.apache.lucene.search.MatchNoDocsQuery;
 import org.apache.lucene.search.MultiTermQuery;
 import org.apache.lucene.search.MultiTermQuery.RewriteMethod;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.search.SynonymQuery;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.spans.SpanBoostQuery;
 import org.apache.lucene.search.spans.SpanNearQuery;
@@ -257,6 +259,7 @@ public class ComplexPhraseQueryParser extends QueryParser {
       // ArrayList spanClauses = new ArrayList();
       if (contents instanceof TermQuery 
           || contents instanceof MultiTermQuery
+          || contents instanceof SynonymQuery
           ) {
         return contents;
       }
@@ -287,9 +290,11 @@ public class ComplexPhraseQueryParser extends QueryParser {
           qc = ((BoostQuery) qc).getQuery();
         }
 
-        if (qc instanceof BooleanQuery) {
+        if (qc instanceof BooleanQuery || qc instanceof SynonymQuery) {
           ArrayList<SpanQuery> sc = new ArrayList<>();
-          addComplexPhraseClause(sc, (BooleanQuery) qc);
+          BooleanQuery booleanCaluse = qc instanceof BooleanQuery ?
+              (BooleanQuery) qc : convert((SynonymQuery) qc);
+          addComplexPhraseClause(sc, booleanCaluse);
           if (sc.size() > 0) {
             allSpanClauses[i] = sc.get(0);
           } else {
@@ -309,14 +314,14 @@ public class ComplexPhraseQueryParser extends QueryParser {
           if (qc instanceof TermQuery) {
             TermQuery tq = (TermQuery) qc;
             allSpanClauses[i] = new SpanTermQuery(tq.getTerm());
-          } else {
+            } else { 
             throw new IllegalArgumentException("Unknown query type \""
                 + qc.getClass().getName()
                 + "\" found in phrase query string \""
                 + phrasedQueryStringContents + "\"");
           }
-
         }
+
         i += 1;
       }
       if (numNegatives == 0) {
@@ -354,6 +359,14 @@ public class ComplexPhraseQueryParser extends QueryParser {
       return snot;
     }
 
+    private BooleanQuery convert(SynonymQuery qc) {
+      BooleanQuery.Builder bqb = new BooleanQuery.Builder();
+      for (Term t : qc.getTerms()){
+        bqb.add(new BooleanClause(new TermQuery(t), Occur.SHOULD));
+      }
+      return bqb.build();
+    }
+
     private void addComplexPhraseClause(List<SpanQuery> spanClauses, BooleanQuery qc) {
       ArrayList<SpanQuery> ors = new ArrayList<>();
       ArrayList<SpanQuery> nots = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8a549293/lucene/queryparser/src/test/org/apache/lucene/queryparser/complexPhrase/TestComplexPhraseQuery.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/complexPhrase/TestComplexPhraseQuery.java b/lucene/queryparser/src/test/org/apache/lucene/queryparser/complexPhrase/TestComplexPhraseQuery.java
index 28b600b..5c45e28 100644
--- a/lucene/queryparser/src/test/org/apache/lucene/queryparser/complexPhrase/TestComplexPhraseQuery.java
+++ b/lucene/queryparser/src/test/org/apache/lucene/queryparser/complexPhrase/TestComplexPhraseQuery.java
@@ -20,6 +20,7 @@ import java.util.HashSet;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.analysis.MockSynonymAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.index.DirectoryReader;
@@ -39,7 +40,11 @@ public class TestComplexPhraseQuery extends LuceneTestCase {
       new DocData("john smith", "1", "developer"),
       new DocData("johathon smith", "2", "developer"),
       new DocData("john percival smith", "3", "designer"),
-      new DocData("jackson waits tom", "4", "project manager")
+      new DocData("jackson waits tom", "4", "project manager"),
+      new DocData("johny perkins", "5", "orders pizza"),
+      new DocData("hapax neverson", "6", "never matches"),
+      new DocData("dog cigar", "7", "just for synonyms"),
+      new DocData("dogs don't smoke cigarettes", "8", "just for synonyms"),
   };
 
   private IndexSearcher searcher;
@@ -73,12 +78,30 @@ public class TestComplexPhraseQuery extends LuceneTestCase {
   }
 
   public void testSingleTermPhrase() throws Exception {
-    checkMatches("\"joh*\" \"tom\"", "1,2,3,4"); 
+    checkMatches("\"joh*\"","1,2,3,5");
+    checkMatches("\"joh~\"","1,3,5");
+    checkMatches("\"joh*\" \"tom\"", "1,2,3,4,5"); 
     checkMatches("+\"j*\" +\"tom\"", "4"); 
-    checkMatches("\"jo*\" \"[sma TO smZ]\" ", "1,2,3");
+    checkMatches("\"jo*\" \"[sma TO smZ]\" ", "1,2,3,5,8");
     checkMatches("+\"j*hn\" +\"sm*h\"", "1,3"); 
   }
 
+  public void testSynonyms() throws Exception {
+    checkMatches("\"dogs\"","8");
+    MockSynonymAnalyzer synonym = new MockSynonymAnalyzer();
+    checkMatches("\"dogs\"","7,8",synonym);
+    // synonym is unidirectional 
+    checkMatches("\"dog\"","7",synonym);
+    checkMatches("\"dogs cigar*\"","");
+    checkMatches("\"dog cigar*\"","7");
+    checkMatches("\"dogs cigar*\"","7", synonym);
+    checkMatches("\"dog cigar*\"","7", synonym);
+    checkMatches("\"dogs cigar*\"~2","7,8", synonym);
+    // synonym is unidirectional
+    checkMatches("\"dog cigar*\"~2","7", synonym);
+    
+  }
+  
   public void testUnOrderedProximitySearches() throws Exception {
 
     inOrder = true;
@@ -98,8 +121,13 @@ public class TestComplexPhraseQuery extends LuceneTestCase {
   }
 
   private void checkMatches(String qString, String expectedVals)
+  throws Exception {
+    checkMatches(qString, expectedVals, analyzer);
+  }
+
+  private void checkMatches(String qString, String expectedVals, Analyzer anAnalyzer)
       throws Exception {
-    ComplexPhraseQueryParser qp = new ComplexPhraseQueryParser(defaultFieldName, analyzer);
+    ComplexPhraseQueryParser qp = new ComplexPhraseQueryParser(defaultFieldName, anAnalyzer);
     qp.setInOrder(inOrder);
     qp.setFuzzyPrefixLength(1); // usually a good idea