You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by kr...@apache.org on 2017/01/03 18:48:12 UTC

[01/50] lucene-solr:jira/solr-8593: SOLR-9787, SOLR-9442: Replace json.nl=arrnvp with json.nl=arrntv (array of Name Type Value) style in JSONResponseWriter

Repository: lucene-solr
Updated Branches:
  refs/heads/jira/solr-8593 918fd97fd -> 82350229c


SOLR-9787, SOLR-9442: Replace json.nl=arrnvp with json.nl=arrntv (array of Name Type Value) style in JSONResponseWriter


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

Branch: refs/heads/jira/solr-8593
Commit: e4ef4239f1b23afb116868e8528f1cd947287bd9
Parents: c2292fa
Author: Christine Poerschke <cp...@apache.org>
Authored: Wed Dec 28 10:41:17 2016 +0000
Committer: Christine Poerschke <cp...@apache.org>
Committed: Wed Dec 28 13:58:55 2016 +0000

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  4 +-
 .../solr/response/JSONResponseWriter.java       | 86 +++++++++++---------
 .../apache/solr/response/JSONWriterTest.java    | 20 +++--
 3 files changed, 60 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e4ef4239/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index f587109..7a708a6 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -130,8 +130,8 @@ New Features
   Example:  { type:terms, field:category, filter:"user:yonik" } 
   (yonik)
 
-* SOLR-9442: Adds Array of NamedValuePair (json.nl=arrnvp) style to JSONResponseWriter.
-  (Jonny Marks, Christine Poerschke)
+* SOLR-9442, SOLR-9787: Adds Array of Name Type Value (json.nl=arrntv) style to JSONResponseWriter.
+  (Jonny Marks, Christine Poerschke, hossman)
 
 * SOLR-9481: Authentication and Authorization plugins now work in standalone mode if security.json is placed in
   SOLR_HOME on every node. Editing config through API is supported but affects only that one node. 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e4ef4239/solr/core/src/java/org/apache/solr/response/JSONResponseWriter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/response/JSONResponseWriter.java b/solr/core/src/java/org/apache/solr/response/JSONResponseWriter.java
index ae1ea47..513df4e 100644
--- a/solr/core/src/java/org/apache/solr/response/JSONResponseWriter.java
+++ b/solr/core/src/java/org/apache/solr/response/JSONResponseWriter.java
@@ -59,9 +59,9 @@ public class JSONResponseWriter implements QueryResponseWriter {
     final String namedListStyle = params.get(JSONWriter.JSON_NL_STYLE, JSONWriter.JSON_NL_FLAT).intern();
 
     final JSONWriter w;
-    if (namedListStyle.equals(JSONWriter.JSON_NL_ARROFNVP)) {
-      w = new ArrayOfNamedValuePairJSONWriter(
-          writer, req, rsp, wrapperFunction, namedListStyle);
+    if (namedListStyle.equals(JSONWriter.JSON_NL_ARROFNTV)) {
+      w = new ArrayOfNameTypeValueJSONWriter(
+          writer, req, rsp, wrapperFunction, namedListStyle, true);
     } else {
       w = new JSONWriter(
           writer, req, rsp, wrapperFunction, namedListStyle);
@@ -96,7 +96,7 @@ class JSONWriter extends TextResponseWriter {
   static final String JSON_NL_FLAT="flat";
   static final String JSON_NL_ARROFARR="arrarr";
   static final String JSON_NL_ARROFMAP="arrmap";
-  static final String JSON_NL_ARROFNVP="arrnvp";
+  static final String JSON_NL_ARROFNTV="arrntv";
 
   static final String JSON_WRAPPER_FUNCTION="json.wrf";
 
@@ -331,9 +331,9 @@ class JSONWriter extends TextResponseWriter {
       writeNamedListAsArrArr(name,val);
     } else if (namedListStyle==JSON_NL_ARROFMAP) {
       writeNamedListAsArrMap(name,val);
-    } else if (namedListStyle==JSON_NL_ARROFNVP) {
+    } else if (namedListStyle==JSON_NL_ARROFNTV) {
       throw new UnsupportedOperationException(namedListStyle
-          + " namedListStyle must only be used with "+ArrayOfNamedValuePairJSONWriter.class.getSimpleName());
+          + " namedListStyle must only be used with "+ArrayOfNameTypeValueJSONWriter.class.getSimpleName());
     }
   }
 
@@ -675,20 +675,25 @@ class JSONWriter extends TextResponseWriter {
 }
 
 /**
- * Writes NamedLists directly as an array of NamedValuePair JSON objects...
- * NamedList("a"=1,"b"=2,null=3,null=null) => [{"name":"a","int":1},{"name":"b","int":2},{"int":3},{"null":null}]
- * NamedList("a"=1,"bar"="foo",null=3.4f) => [{"name":"a","int":1},{"name":"bar","str":"foo"},{"float":3.4}]
+ * Writes NamedLists directly as an array of NameTypeValue JSON objects...
+ * NamedList("a"=1,"b"=null,null=3,null=null) =>
+ *      [{"name":"a","type":"int","value":1},
+ *       {"name":"b","type":"null","value":null},
+ *       {"name":null,"type":"int","value":3},
+ *       {"name":null,"type":"null","value":null}]
+ * NamedList("a"=1,"bar"="foo",null=3.4f) =>
+ *      [{"name":"a","type":"int","value":1},
+ *      {"name":"bar","type":"str","value":"foo"},
+ *      {"name":null,"type":"float","value":3.4}]
  */
-class ArrayOfNamedValuePairJSONWriter extends JSONWriter {
-  private boolean writeTypeAsKey = false;
+class ArrayOfNameTypeValueJSONWriter extends JSONWriter {
+  protected boolean writeTypeAndValueKey = false;
+  private final boolean writeNullName;
 
-  public ArrayOfNamedValuePairJSONWriter(Writer writer, SolrQueryRequest req, SolrQueryResponse rsp,
-                                         String wrapperFunction, String namedListStyle) {
+  public ArrayOfNameTypeValueJSONWriter(Writer writer, SolrQueryRequest req, SolrQueryResponse rsp,
+                                        String wrapperFunction, String namedListStyle, boolean writeNullName) {
     super(writer, req, rsp, wrapperFunction, namedListStyle);
-    if (namedListStyle != JSON_NL_ARROFNVP) {
-      throw new UnsupportedOperationException(ArrayOfNamedValuePairJSONWriter.class.getSimpleName()+" must only be used with "
-          + JSON_NL_ARROFNVP + " style");
-    }
+    this.writeNullName = writeNullName;
   }
 
   @Override
@@ -720,24 +725,24 @@ class ArrayOfNamedValuePairJSONWriter extends JSONWriter {
 
       /*
        * JSONWriter's writeNamedListAsArrMap turns NamedList("bar"="foo") into [{"foo":"bar"}]
-       * but we here wish to turn it into [ {"name":"bar","str":"foo"} ] instead.
+       * but we here wish to turn it into [ {"name":"bar","type":"str","value":"foo"} ] instead.
        *
        * So first we write the <code>{"name":"bar",</code> portion ...
        */
       writeMapOpener(-1);
-      if (elementName != null) {
+      if (elementName != null || writeNullName) {
         writeKey("name", false);
         writeVal("name", elementName);
         writeMapSeparator();
       }
 
       /*
-       * ... and then we write the <code>"str":"foo"}</code> portion.
+       * ... and then we write the <code>"type":"str","value":"foo"}</code> portion.
        */
-      writeTypeAsKey = true;
+      writeTypeAndValueKey = true;
       writeVal(null, elementVal); // passing null since writeVal doesn't actually use name (and we already wrote elementName above)
-      if (writeTypeAsKey) {
-        throw new RuntimeException("writeTypeAsKey should have been reset to false by writeVal('"+elementName+"','"+elementVal+"')");
+      if (writeTypeAndValueKey) {
+        throw new RuntimeException("writeTypeAndValueKey should have been reset to false by writeVal('"+elementName+"','"+elementVal+"')");
       }
       writeMapCloser();
     }
@@ -746,82 +751,85 @@ class ArrayOfNamedValuePairJSONWriter extends JSONWriter {
     writeArrayCloser();
   }
 
-  private void ifNeededWriteTypeAsKey(String type) throws IOException {
-    if (writeTypeAsKey) {
-      writeTypeAsKey = false;
-      writeKey(type, false);
+  protected void ifNeededWriteTypeAndValueKey(String type) throws IOException {
+    if (writeTypeAndValueKey) {
+      writeTypeAndValueKey = false;
+      writeKey("type", false);
+      writeVal("type", type);
+      writeMapSeparator();
+      writeKey("value", false);
     }
   }
 
   @Override
   public void writeInt(String name, String val) throws IOException {
-    ifNeededWriteTypeAsKey("int");
+    ifNeededWriteTypeAndValueKey("int");
     super.writeInt(name, val);
   }
 
   @Override
   public void writeLong(String name, String val) throws IOException {
-    ifNeededWriteTypeAsKey("long");
+    ifNeededWriteTypeAndValueKey("long");
     super.writeLong(name, val);
   }
 
   @Override
   public void writeFloat(String name, String val) throws IOException {
-    ifNeededWriteTypeAsKey("float");
+    ifNeededWriteTypeAndValueKey("float");
     super.writeFloat(name, val);
   }
 
   @Override
   public void writeDouble(String name, String val) throws IOException {
-    ifNeededWriteTypeAsKey("double");
+    ifNeededWriteTypeAndValueKey("double");
     super.writeDouble(name, val);
   }
 
   @Override
   public void writeBool(String name, String val) throws IOException {
-    ifNeededWriteTypeAsKey("bool");
+    ifNeededWriteTypeAndValueKey("bool");
     super.writeBool(name, val);
   }
 
   @Override
   public void writeDate(String name, String val) throws IOException {
-    ifNeededWriteTypeAsKey("date");
+    ifNeededWriteTypeAndValueKey("date");
     super.writeDate(name, val);
   }
 
   @Override
   public void writeStr(String name, String val, boolean needsEscaping) throws IOException {
-    ifNeededWriteTypeAsKey("str");
+    ifNeededWriteTypeAndValueKey("str");
     super.writeStr(name, val, needsEscaping);
   }
 
   @Override
   public void writeSolrDocument(String name, SolrDocument doc, ReturnFields returnFields, int idx) throws IOException {
-    ifNeededWriteTypeAsKey("doc");
+    ifNeededWriteTypeAndValueKey("doc");
     super.writeSolrDocument(name, doc, returnFields, idx);
   }
 
   @Override
   public void writeStartDocumentList(String name, long start, int size, long numFound, Float maxScore) throws IOException {
-    ifNeededWriteTypeAsKey("doclist");
+    ifNeededWriteTypeAndValueKey("doclist");
     super.writeStartDocumentList(name, start, size, numFound, maxScore);
   }
 
   @Override
   public void writeMap(String name, Map val, boolean excludeOuter, boolean isFirstVal) throws IOException {
-    ifNeededWriteTypeAsKey("map");
+    ifNeededWriteTypeAndValueKey("map");
     super.writeMap(name, val, excludeOuter, isFirstVal);
   }
 
   @Override
   public void writeArray(String name, Iterator val) throws IOException {
-    ifNeededWriteTypeAsKey("array");
+    ifNeededWriteTypeAndValueKey("array");
     super.writeArray(name, val);
   }
 
   @Override
   public void writeNull(String name) throws IOException {
-    ifNeededWriteTypeAsKey("null");
+    ifNeededWriteTypeAndValueKey("null");
     super.writeNull(name);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e4ef4239/solr/core/src/test/org/apache/solr/response/JSONWriterTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/response/JSONWriterTest.java b/solr/core/src/test/org/apache/solr/response/JSONWriterTest.java
index a056016..45ca708 100644
--- a/solr/core/src/test/org/apache/solr/response/JSONWriterTest.java
+++ b/solr/core/src/test/org/apache/solr/response/JSONWriterTest.java
@@ -81,7 +81,7 @@ public class JSONWriterTest extends SolrTestCaseJ4 {
         JSONWriter.JSON_NL_MAP,
         JSONWriter.JSON_NL_ARROFARR,
         JSONWriter.JSON_NL_ARROFMAP,
-        JSONWriter.JSON_NL_ARROFNVP,
+        JSONWriter.JSON_NL_ARROFNTV,
     };
     for (final String namedListStyle : namedListStyles) {
       implTestJSON(namedListStyle);
@@ -116,8 +116,10 @@ public class JSONWriterTest extends SolrTestCaseJ4 {
       expectedNLjson = "\"nl\":[[\"data1\",\"he\\u2028llo\\u2029!\"],[null,42],[null,null]]";
     } else if (namedListStyle == JSONWriter.JSON_NL_ARROFMAP) {
       expectedNLjson = "\"nl\":[{\"data1\":\"he\\u2028llo\\u2029!\"},42,null]";
-    } else if (namedListStyle == JSONWriter.JSON_NL_ARROFNVP) {
-      expectedNLjson = "\"nl\":[{\"name\":\"data1\",\"str\":\"he\\u2028llo\\u2029!\"},{\"int\":42},{\"null\":null}]";
+    } else if (namedListStyle == JSONWriter.JSON_NL_ARROFNTV) {
+      expectedNLjson = "\"nl\":[{\"name\":\"data1\",\"type\":\"str\",\"value\":\"he\\u2028llo\\u2029!\"}," +
+          "{\"name\":null,\"type\":\"int\",\"value\":42}," +
+          "{\"name\":null,\"type\":\"null\",\"value\":null}]";
     } else {
       expectedNLjson = null;
       fail("unexpected namedListStyle="+namedListStyle);
@@ -168,7 +170,7 @@ public class JSONWriterTest extends SolrTestCaseJ4 {
   }
 
   @Test
-  public void testArrnvpWriterOverridesAllWrites() {
+  public void testArrntvWriterOverridesAllWrites() {
     // List rather than Set because two not-overridden methods could share name but not signature
     final List<String> methodsExpectedNotOverriden = new ArrayList<>(14);
     methodsExpectedNotOverriden.add("writeResponse");
@@ -189,7 +191,7 @@ public class JSONWriterTest extends SolrTestCaseJ4 {
     methodsExpectedNotOverriden.add("public void org.apache.solr.response.JSONWriter.writeMap(org.apache.solr.common.MapWriter) throws java.io.IOException");
     methodsExpectedNotOverriden.add("public void org.apache.solr.response.JSONWriter.writeIterator(org.apache.solr.common.IteratorWriter) throws java.io.IOException");
 
-    final Class<?> subClass = ArrayOfNamedValuePairJSONWriter.class;
+    final Class<?> subClass = ArrayOfNameTypeValueJSONWriter.class;
     final Class<?> superClass = subClass.getSuperclass();
 
     for (final Method superClassMethod : superClass.getDeclaredMethods()) {
@@ -231,14 +233,14 @@ public class JSONWriterTest extends SolrTestCaseJ4 {
   }
 
   @Test
-  public void testArrnvpWriterLacksMethodsOfItsOwn() {
-    final Class<?> subClass = ArrayOfNamedValuePairJSONWriter.class;
+  public void testArrntvWriterLacksMethodsOfItsOwn() {
+    final Class<?> subClass = ArrayOfNameTypeValueJSONWriter.class;
     final Class<?> superClass = subClass.getSuperclass();
     // ArrayOfNamedValuePairJSONWriter is a simple sub-class
     // which should have (almost) no methods of its own
     for (final Method subClassMethod : subClass.getDeclaredMethods()) {
       // only own private method of its own
-      if (subClassMethod.getName().equals("ifNeededWriteTypeAsKey")) continue;
+      if (subClassMethod.getName().equals("ifNeededWriteTypeAndValueKey")) continue;
       try {
         final Method superClassMethod = superClass.getDeclaredMethod(
             subClassMethod.getName(),
@@ -260,7 +262,7 @@ public class JSONWriterTest extends SolrTestCaseJ4 {
     assertEquals("flat", JSONWriter.JSON_NL_FLAT);
     assertEquals("arrarr", JSONWriter.JSON_NL_ARROFARR);
     assertEquals("arrmap", JSONWriter.JSON_NL_ARROFMAP);
-    assertEquals("arrnvp", JSONWriter.JSON_NL_ARROFNVP);
+    assertEquals("arrntv", JSONWriter.JSON_NL_ARROFNTV);
     assertEquals("json.wrf", JSONWriter.JSON_WRAPPER_FUNCTION);
   }
 


[32/50] lucene-solr:jira/solr-8593: SOLR-9684: Update CHANGES.txt

Posted by kr...@apache.org.
SOLR-9684: Update CHANGES.txt


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

Branch: refs/heads/jira/solr-8593
Commit: 36a691c50d680d1c6977e6185448e06cb21f653d
Parents: f3fe487
Author: Joel Bernstein <jb...@apache.org>
Authored: Fri Dec 30 15:38:28 2016 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Fri Dec 30 15:38:28 2016 -0500

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


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/36a691c5/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 874ac81..6a99617 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -208,6 +208,8 @@ New Features
 
 * SOLR-9668,SOLR-7197: introduce cursorMark='true' in SolrEntityProcessor (Yegor Kozlov, Raveendra Yerraguntl via Mikhail Khludnev)
 
+* SOLR-9684: Add schedule Streaming Expression (Joel Bernstein)
+
 Optimizations
 ----------------------
 * SOLR-9704: Facet Module / JSON Facet API: Optimize blockChildren facets that have


[37/50] lucene-solr:jira/solr-8593: SOLR-9684: Rename schedule function to priority

Posted by kr...@apache.org.
SOLR-9684: Rename schedule function to priority


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

Branch: refs/heads/jira/solr-8593
Commit: 0999f6779a3341af072d31162a2c88cf1eb8c5d4
Parents: 93562da
Author: Joel Bernstein <jb...@apache.org>
Authored: Mon Jan 2 11:08:44 2017 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Mon Jan 2 11:08:44 2017 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   2 +-
 .../org/apache/solr/handler/StreamHandler.java  |   2 +-
 .../client/solrj/io/stream/PriorityStream.java  | 161 +++++++++++++++++++
 .../client/solrj/io/stream/SchedulerStream.java | 161 -------------------
 .../solrj/io/stream/StreamExpressionTest.java   |  20 +--
 5 files changed, 173 insertions(+), 173 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0999f677/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 7133638..6326e54 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -208,7 +208,7 @@ New Features
 
 * SOLR-9668,SOLR-7197: introduce cursorMark='true' in SolrEntityProcessor (Yegor Kozlov, Raveendra Yerraguntl via Mikhail Khludnev)
 
-* SOLR-9684: Add schedule Streaming Expression (Joel Bernstein)
+* SOLR-9684: Add priority Streaming Expression (Joel Bernstein, David Smiley)
 
 Optimizations
 ----------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0999f677/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 1610fea..661704f 100644
--- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
@@ -140,7 +140,7 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
       .withFunctionName("fetch", FetchStream.class)
       .withFunctionName("executor", ExecutorStream.class)
       .withFunctionName("null", NullStream.class)
-      .withFunctionName("schedule", SchedulerStream.class)
+      .withFunctionName("priority", PriorityStream.class)
       // metrics
       .withFunctionName("min", MinMetric.class)
       .withFunctionName("max", MaxMetric.class)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0999f677/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/PriorityStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/PriorityStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/PriorityStream.java
new file mode 100644
index 0000000..c5faf41
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/PriorityStream.java
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.client.solrj.io.stream;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+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.comp.StreamComparator;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType;
+import org.apache.solr.client.solrj.io.stream.expr.Expressible;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExplanation;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The priority function wraps two topics that represent high priority and low priority task queues.
+ * Each time the priority function is called it will check to see if there are any high priority tasks in the queue. If there
+ * are high priority tasks, then the high priority queue will be read until it returns the EOF Tuple.
+ *
+ * If there are no tasks in the high priority queue, then the lower priority task queue will be opened and read until the EOF Tuple is
+ * returned.
+ *
+ * The scheduler is designed to be wrapped by the executor function and a daemon function can be used to call the executor iteratively.
+ **/
+
+public class PriorityStream extends TupleStream implements Expressible {
+
+  private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private PushBackStream highPriorityTasks;
+  private PushBackStream tasks;
+  private TupleStream currentStream;
+
+  public PriorityStream(StreamExpression expression, StreamFactory factory) throws IOException {
+    // grab all parameters out
+    List<StreamExpression> streamExpressions = factory.getExpressionOperandsRepresentingTypes(expression, Expressible.class, TupleStream.class);
+
+
+    if(2 != streamExpressions.size()){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting a single stream but found %d",expression, streamExpressions.size()));
+    }
+
+    TupleStream stream1 = factory.constructStream(streamExpressions.get(0));
+    TupleStream stream2 = factory.constructStream(streamExpressions.get(1));
+
+    if(!(stream1 instanceof TopicStream) || !(stream2 instanceof TopicStream)) {
+      throw new IOException("The scheduler expects both stream parameters to be topics.");
+    }
+
+    init(new PushBackStream(stream1), new PushBackStream(stream2));
+  }
+
+  private void init(PushBackStream stream1, PushBackStream stream2) throws IOException{
+    this.highPriorityTasks = stream1;
+    this.tasks = stream2;
+  }
+
+  @Override
+  public StreamExpression toExpression(StreamFactory factory) throws IOException {
+    return toExpression(factory, true);
+  }
+
+  private StreamExpression toExpression(StreamFactory factory, boolean includeStreams) throws IOException {
+
+    // function name
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
+
+    // stream
+    if(includeStreams) {
+      if (highPriorityTasks instanceof Expressible) {
+        expression.addParameter(((Expressible) highPriorityTasks).toExpression(factory));
+      } else {
+        throw new IOException("The SchedulerStream contains a non-expressible TupleStream - it cannot be converted to an expression");
+      }
+
+      if (tasks instanceof Expressible) {
+        expression.addParameter(((Expressible) tasks).toExpression(factory));
+      } else {
+        throw new IOException("The SchedulerStream contains a non-expressible TupleStream - it cannot be converted to an expression");
+      }
+    }
+
+    return expression;
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+
+    return new StreamExplanation(getStreamNodeId().toString())
+        .withChildren(new Explanation[]{
+            highPriorityTasks.toExplanation(factory), tasks.toExplanation(factory)
+        })
+        .withFunctionName(factory.getFunctionName(this.getClass()))
+        .withImplementingClass(this.getClass().getName())
+        .withExpressionType(ExpressionType.STREAM_DECORATOR)
+        .withExpression(toExpression(factory, false).toString());
+  }
+
+  public void setStreamContext(StreamContext streamContext) {
+    this.highPriorityTasks.setStreamContext(streamContext);
+    tasks.setStreamContext(streamContext);
+  }
+
+  public List<TupleStream> children() {
+    List<TupleStream> l =  new ArrayList();
+    l.add(highPriorityTasks);
+    l.add(tasks);
+    return l;
+  }
+
+  public void open() throws IOException {
+    highPriorityTasks.open();
+    Tuple tuple = highPriorityTasks.read();
+    if(tuple.EOF) {
+      highPriorityTasks.close();
+      tasks.open();
+      currentStream = tasks;
+    } else {
+      highPriorityTasks.pushBack(tuple);
+      currentStream = highPriorityTasks;
+    }
+  }
+
+  public void close() throws IOException {
+      currentStream.close();
+  }
+
+  public Tuple read() throws IOException {
+    return currentStream.read();
+  }
+
+  public StreamComparator getStreamSort(){
+    return null;
+  }
+
+  public int getCost() {
+    return 0;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0999f677/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SchedulerStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SchedulerStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SchedulerStream.java
deleted file mode 100644
index f8506b9..0000000
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SchedulerStream.java
+++ /dev/null
@@ -1,161 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.solr.client.solrj.io.stream;
-
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-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.comp.StreamComparator;
-import org.apache.solr.client.solrj.io.stream.expr.Explanation;
-import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType;
-import org.apache.solr.client.solrj.io.stream.expr.Expressible;
-import org.apache.solr.client.solrj.io.stream.expr.StreamExplanation;
-import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
-import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * The scheduler wraps two topics that represent high priority and low priority task queues.
- * Each time the scheduler is called it will check to see if there are any high priority tasks in the queue. If there
- * are high priority tasks, then the high priority queue will be read until it returns the EOF Tuple.
- *
- * If there are no tasks in the high priority queue, then the lower priority task queue will be opened and read until the EOF Tuple is
- * returned.
- *
- * The scheduler is designed to be wrapped by the executor function and a daemon function can be used to call the executor iteratively.
- **/
-
-public class SchedulerStream extends TupleStream implements Expressible {
-
-  private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-  private PushBackStream highPriorityTasks;
-  private PushBackStream tasks;
-  private TupleStream currentStream;
-
-  public SchedulerStream(StreamExpression expression, StreamFactory factory) throws IOException {
-    // grab all parameters out
-    List<StreamExpression> streamExpressions = factory.getExpressionOperandsRepresentingTypes(expression, Expressible.class, TupleStream.class);
-
-
-    if(2 != streamExpressions.size()){
-      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting a single stream but found %d",expression, streamExpressions.size()));
-    }
-
-    TupleStream stream1 = factory.constructStream(streamExpressions.get(0));
-    TupleStream stream2 = factory.constructStream(streamExpressions.get(1));
-
-    if(!(stream1 instanceof TopicStream) || !(stream2 instanceof TopicStream)) {
-      throw new IOException("The scheduler expects both stream parameters to be topics.");
-    }
-
-    init(new PushBackStream(stream1), new PushBackStream(stream2));
-  }
-
-  private void init(PushBackStream stream1, PushBackStream stream2) throws IOException{
-    this.highPriorityTasks = stream1;
-    this.tasks = stream2;
-  }
-
-  @Override
-  public StreamExpression toExpression(StreamFactory factory) throws IOException {
-    return toExpression(factory, true);
-  }
-
-  private StreamExpression toExpression(StreamFactory factory, boolean includeStreams) throws IOException {
-
-    // function name
-    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
-
-    // stream
-    if(includeStreams) {
-      if (highPriorityTasks instanceof Expressible) {
-        expression.addParameter(((Expressible) highPriorityTasks).toExpression(factory));
-      } else {
-        throw new IOException("The SchedulerStream contains a non-expressible TupleStream - it cannot be converted to an expression");
-      }
-
-      if (tasks instanceof Expressible) {
-        expression.addParameter(((Expressible) tasks).toExpression(factory));
-      } else {
-        throw new IOException("The SchedulerStream contains a non-expressible TupleStream - it cannot be converted to an expression");
-      }
-    }
-
-    return expression;
-  }
-
-  @Override
-  public Explanation toExplanation(StreamFactory factory) throws IOException {
-
-    return new StreamExplanation(getStreamNodeId().toString())
-        .withChildren(new Explanation[]{
-            highPriorityTasks.toExplanation(factory), tasks.toExplanation(factory)
-        })
-        .withFunctionName(factory.getFunctionName(this.getClass()))
-        .withImplementingClass(this.getClass().getName())
-        .withExpressionType(ExpressionType.STREAM_DECORATOR)
-        .withExpression(toExpression(factory, false).toString());
-  }
-
-  public void setStreamContext(StreamContext streamContext) {
-    this.highPriorityTasks.setStreamContext(streamContext);
-    tasks.setStreamContext(streamContext);
-  }
-
-  public List<TupleStream> children() {
-    List<TupleStream> l =  new ArrayList();
-    l.add(highPriorityTasks);
-    l.add(tasks);
-    return l;
-  }
-
-  public void open() throws IOException {
-    highPriorityTasks.open();
-    Tuple tuple = highPriorityTasks.read();
-    if(tuple.EOF) {
-      highPriorityTasks.close();
-      tasks.open();
-      currentStream = tasks;
-    } else {
-      highPriorityTasks.pushBack(tuple);
-      currentStream = highPriorityTasks;
-    }
-  }
-
-  public void close() throws IOException {
-      currentStream.close();
-  }
-
-  public Tuple read() throws IOException {
-    return currentStream.read();
-  }
-
-  public StreamComparator getStreamSort(){
-    return null;
-  }
-
-  public int getCost() {
-    return 0;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0999f677/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 936d42f..1316af4 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
@@ -2826,7 +2826,7 @@ public class StreamExpressionTest extends SolrCloudTestCase {
   }
 
   @Test
-  public void testSchedulerStream() throws Exception {
+  public void testPriorityStream() throws Exception {
     Assume.assumeTrue(!useAlias);
 
     new UpdateRequest()
@@ -2845,7 +2845,7 @@ public class StreamExpressionTest extends SolrCloudTestCase {
     StreamFactory factory = new StreamFactory()
         .withCollectionZkHost("collection1", cluster.getZkServer().getZkAddress())
         .withFunctionName("topic", TopicStream.class)
-        .withFunctionName("schedule", SchedulerStream.class);
+        .withFunctionName("priority", PriorityStream.class);
 
     StreamExpression expression;
     TupleStream stream;
@@ -2856,7 +2856,7 @@ public class StreamExpressionTest extends SolrCloudTestCase {
     try {
       FieldComparator comp = new FieldComparator("a_i", ComparatorOrder.ASCENDING);
 
-      expression = StreamExpressionParser.parse("schedule(topic(collection1, collection1, q=\"a_s:hello\", fl=\"id,a_i\", id=1000000, initialCheckpoint=0)," +
+      expression = StreamExpressionParser.parse("priority(topic(collection1, collection1, q=\"a_s:hello\", fl=\"id,a_i\", id=1000000, initialCheckpoint=0)," +
           "topic(collection1, collection1, q=\"a_s:hello1\", fl=\"id,a_i\", id=2000000, initialCheckpoint=0))");
       stream = factory.constructStream(expression);
       StreamContext context = new StreamContext();
@@ -2870,7 +2870,7 @@ public class StreamExpressionTest extends SolrCloudTestCase {
       assertEquals(tuples.size(), 4);
       assertOrder(tuples, 5, 6, 7, 8);
 
-      expression = StreamExpressionParser.parse("schedule(topic(collection1, collection1, q=\"a_s:hello\", fl=\"id,a_i\", id=1000000, initialCheckpoint=0)," +
+      expression = StreamExpressionParser.parse("priority(topic(collection1, collection1, q=\"a_s:hello\", fl=\"id,a_i\", id=1000000, initialCheckpoint=0)," +
           "topic(collection1, collection1, q=\"a_s:hello1\", fl=\"id,a_i\", id=2000000, initialCheckpoint=0))");
       stream = factory.constructStream(expression);
       context = new StreamContext();
@@ -2883,7 +2883,7 @@ public class StreamExpressionTest extends SolrCloudTestCase {
       assertEquals(tuples.size(), 6);
       assertOrder(tuples, 0, 1, 2, 3, 4, 9);
 
-      expression = StreamExpressionParser.parse("schedule(topic(collection1, collection1, q=\"a_s:hello\", fl=\"id,a_i\", id=1000000, initialCheckpoint=0)," +
+      expression = StreamExpressionParser.parse("priority(topic(collection1, collection1, q=\"a_s:hello\", fl=\"id,a_i\", id=1000000, initialCheckpoint=0)," +
           "topic(collection1, collection1, q=\"a_s:hello1\", fl=\"id,a_i\", id=2000000, initialCheckpoint=0))");
       stream = factory.constructStream(expression);
       context = new StreamContext();
@@ -2900,7 +2900,7 @@ public class StreamExpressionTest extends SolrCloudTestCase {
   }
 
   @Test
-  public void testParallelSchedulerStream() throws Exception {
+  public void testParallelPriorityStream() throws Exception {
     Assume.assumeTrue(!useAlias);
 
     new UpdateRequest()
@@ -2920,7 +2920,7 @@ public class StreamExpressionTest extends SolrCloudTestCase {
         .withCollectionZkHost("collection1", cluster.getZkServer().getZkAddress())
         .withFunctionName("topic", TopicStream.class)
         .withFunctionName("parallel", ParallelStream.class)
-        .withFunctionName("schedule", SchedulerStream.class);
+        .withFunctionName("priority", PriorityStream.class);
 
     StreamExpression expression;
     TupleStream stream;
@@ -2931,7 +2931,7 @@ public class StreamExpressionTest extends SolrCloudTestCase {
     try {
       FieldComparator comp = new FieldComparator("a_i", ComparatorOrder.ASCENDING);
 
-      expression = StreamExpressionParser.parse("parallel(collection1, workers=2, sort=\"_version_ asc\", schedule(topic(collection1, collection1, q=\"a_s:hello\", fl=\"id,a_i\", id=1000000, initialCheckpoint=0, partitionKeys=id)," +
+      expression = StreamExpressionParser.parse("parallel(collection1, workers=2, sort=\"_version_ asc\", priority(topic(collection1, collection1, q=\"a_s:hello\", fl=\"id,a_i\", id=1000000, initialCheckpoint=0, partitionKeys=id)," +
           "topic(collection1, collection1, q=\"a_s:hello1\", fl=\"id,a_i\", id=2000000, initialCheckpoint=0, partitionKeys=id)))");
       stream = factory.constructStream(expression);
       StreamContext context = new StreamContext();
@@ -2945,7 +2945,7 @@ public class StreamExpressionTest extends SolrCloudTestCase {
       assertEquals(tuples.size(), 4);
       assertOrder(tuples, 5, 6, 7, 8);
 
-      expression = StreamExpressionParser.parse("parallel(collection1, workers=2, sort=\"_version_ asc\", schedule(topic(collection1, collection1, q=\"a_s:hello\", fl=\"id,a_i\", id=1000000, initialCheckpoint=0, partitionKeys=id)," +
+      expression = StreamExpressionParser.parse("parallel(collection1, workers=2, sort=\"_version_ asc\", priority(topic(collection1, collection1, q=\"a_s:hello\", fl=\"id,a_i\", id=1000000, initialCheckpoint=0, partitionKeys=id)," +
           "topic(collection1, collection1, q=\"a_s:hello1\", fl=\"id,a_i\", id=2000000, initialCheckpoint=0, partitionKeys=id)))");
       stream = factory.constructStream(expression);
       context = new StreamContext();
@@ -2958,7 +2958,7 @@ public class StreamExpressionTest extends SolrCloudTestCase {
       assertEquals(tuples.size(), 6);
       assertOrder(tuples, 0, 1, 2, 3, 4, 9);
 
-      expression = StreamExpressionParser.parse("parallel(collection1, workers=2, sort=\"_version_ asc\", schedule(topic(collection1, collection1, q=\"a_s:hello\", fl=\"id,a_i\", id=1000000, initialCheckpoint=0, partitionKeys=id)," +
+      expression = StreamExpressionParser.parse("parallel(collection1, workers=2, sort=\"_version_ asc\", priority(topic(collection1, collection1, q=\"a_s:hello\", fl=\"id,a_i\", id=1000000, initialCheckpoint=0, partitionKeys=id)," +
           "topic(collection1, collection1, q=\"a_s:hello1\", fl=\"id,a_i\", id=2000000, initialCheckpoint=0, partitionKeys=id)))");
       stream = factory.constructStream(expression);
       context = new StreamContext();


[46/50] lucene-solr:jira/solr-8593: SOLR-9854: Collect metrics for index merges and index store IO. (squashed)

Posted by kr...@apache.org.
SOLR-9854: Collect metrics for index merges and index store IO. (squashed)


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

Branch: refs/heads/jira/solr-8593
Commit: 48ca9fc3f4f8d95293cee7bb59eff61247ede181
Parents: 7b2e3db
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Tue Jan 3 11:11:50 2017 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Tue Jan 3 12:12:01 2017 +0100

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   2 +
 .../solr/core/MetricsDirectoryFactory.java      | 511 +++++++++++++++++++
 .../src/java/org/apache/solr/core/SolrCore.java |  24 +-
 .../apache/solr/core/SolrDeletionPolicy.java    |   6 +
 .../org/apache/solr/core/SolrInfoMBean.java     |   2 +-
 .../solr/core/StandardDirectoryFactory.java     |   2 +-
 .../org/apache/solr/update/SolrIndexConfig.java |  16 +-
 .../org/apache/solr/update/SolrIndexWriter.java | 144 ++++++
 .../org/apache/solr/util/stats/MetricUtils.java |  79 ++-
 .../conf/solrconfig-indexmetrics.xml            |  57 +++
 .../test/org/apache/solr/core/TestConfig.java   |   2 +
 .../solr/handler/TestReplicationHandler.java    |  13 +-
 .../admin/CoreMergeIndexesAdminHandlerTest.java |  10 +-
 .../solr/handler/admin/MetricsHandlerTest.java  |   6 +-
 .../apache/solr/update/SolrIndexConfigTest.java |   1 +
 .../solr/update/SolrIndexMetricsTest.java       |  94 ++++
 .../apache/solr/util/stats/MetricUtilsTest.java |  24 +-
 17 files changed, 945 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/48ca9fc3/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 8609f91..afcd295 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -212,6 +212,8 @@ New Features
 
 * SOLR-9896: Instrument and collect metrics from query, update, core admin and core load thread pools. (shalin)
 
+* SOLR-9854: Collect metrics for index merges and index store IO (ab)
+
 Optimizations
 ----------------------
 * SOLR-9704: Facet Module / JSON Facet API: Optimize blockChildren facets that have

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/48ca9fc3/solr/core/src/java/org/apache/solr/core/MetricsDirectoryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/MetricsDirectoryFactory.java b/solr/core/src/java/org/apache/solr/core/MetricsDirectoryFactory.java
new file mode 100644
index 0000000..62e82ac
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/core/MetricsDirectoryFactory.java
@@ -0,0 +1,511 @@
+/*
+ * 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.io.IOException;
+import java.util.Collection;
+
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.Meter;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FilterDirectory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.LockFactory;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.util.plugin.SolrCoreAware;
+
+/**
+ * An implementation of {@link DirectoryFactory} that decorates provided factory by
+ * adding metrics for directory IO operations.
+ */
+public class MetricsDirectoryFactory extends DirectoryFactory implements SolrCoreAware {
+  private final SolrMetricManager metricManager;
+  private final String registry;
+  private final DirectoryFactory in;
+  private boolean directoryDetails = false;
+
+  public MetricsDirectoryFactory(SolrMetricManager metricManager, String registry, DirectoryFactory in) {
+    this.metricManager = metricManager;
+    this.registry = registry;
+    this.in = in;
+  }
+
+  public DirectoryFactory getDelegate() {
+    return in;
+  }
+
+  /**
+   * Currently the following arguments are supported:
+   * <ul>
+   *   <li><code>directoryDetails</code> - (optional bool) when true then additional detailed metrics
+   *   will be collected. These include eg. IO size histograms and per-file counters and histograms</li>
+   * </ul>
+   * @param args init args
+   */
+  @Override
+  public void init(NamedList args) {
+    // should be already inited
+    // in.init(args);
+    if (args == null) {
+      return;
+    }
+    Boolean dd = args.getBooleanArg("directoryDetails");
+    if (dd != null) {
+      directoryDetails = dd;
+    } else {
+      directoryDetails = false;
+    }
+  }
+
+  @Override
+  public void doneWithDirectory(Directory dir) throws IOException {
+    // unwrap
+    if (dir instanceof MetricsDirectory) {
+      dir = ((MetricsDirectory)dir).getDelegate();
+    }
+    in.doneWithDirectory(dir);
+  }
+
+  @Override
+  public void addCloseListener(Directory dir, CachingDirectoryFactory.CloseListener closeListener) {
+    // unwrap
+    if (dir instanceof MetricsDirectory) {
+      dir = ((MetricsDirectory)dir).getDelegate();
+    }
+    in.addCloseListener(dir, closeListener);
+  }
+
+  @Override
+  public void close() throws IOException {
+    in.close();
+  }
+
+  @Override
+  protected Directory create(String path, LockFactory lockFactory, DirContext dirContext) throws IOException {
+    Directory dir = in.create(path, lockFactory, dirContext);
+    return new MetricsDirectory(metricManager, registry, dir, directoryDetails);
+  }
+
+  @Override
+  protected LockFactory createLockFactory(String rawLockType) throws IOException {
+    return in.createLockFactory(rawLockType);
+  }
+
+  @Override
+  public boolean exists(String path) throws IOException {
+    return in.exists(path);
+  }
+
+  @Override
+  public void remove(Directory dir) throws IOException {
+    // unwrap
+    if (dir instanceof MetricsDirectory) {
+      dir = ((MetricsDirectory)dir).getDelegate();
+    }
+    in.remove(dir);
+  }
+
+  @Override
+  public void remove(Directory dir, boolean afterCoreClose) throws IOException {
+    // unwrap
+    if (dir instanceof MetricsDirectory) {
+      dir = ((MetricsDirectory)dir).getDelegate();
+    }
+    in.remove(dir, afterCoreClose);
+  }
+
+  @Override
+  public boolean isSharedStorage() {
+    return in.isSharedStorage();
+  }
+
+  @Override
+  public boolean isAbsolute(String path) {
+    return in.isAbsolute(path);
+  }
+
+  @Override
+  public boolean searchersReserveCommitPoints() {
+    return in.searchersReserveCommitPoints();
+  }
+
+  @Override
+  public String getDataHome(CoreDescriptor cd) throws IOException {
+    return in.getDataHome(cd);
+  }
+
+  @Override
+  public long size(Directory directory) throws IOException {
+    return in.size(directory);
+  }
+
+  @Override
+  public long size(String path) throws IOException {
+    return in.size(path);
+  }
+
+  @Override
+  public Collection<SolrInfoMBean> offerMBeans() {
+    return in.offerMBeans();
+  }
+
+  @Override
+  public void cleanupOldIndexDirectories(String dataDirPath, String currentIndexDirPath) {
+    in.cleanupOldIndexDirectories(dataDirPath, currentIndexDirPath);
+  }
+
+  @Override
+  public void remove(String path, boolean afterCoreClose) throws IOException {
+    in.remove(path, afterCoreClose);
+  }
+
+  @Override
+  public void remove(String path) throws IOException {
+    in.remove(path);
+  }
+
+  @Override
+  public void move(Directory fromDir, Directory toDir, String fileName, IOContext ioContext) throws IOException {
+    in.move(fromDir, toDir, fileName, ioContext);
+  }
+
+  @Override
+  public Directory get(String path, DirContext dirContext, String rawLockType) throws IOException {
+    Directory dir = in.get(path, dirContext, rawLockType);
+    if (dir instanceof MetricsDirectory) {
+      return dir;
+    } else {
+      return new MetricsDirectory(metricManager, registry, dir, directoryDetails);
+    }
+  }
+
+  @Override
+  public void renameWithOverwrite(Directory dir, String fileName, String toName) throws IOException {
+    super.renameWithOverwrite(dir, fileName, toName);
+  }
+
+  @Override
+  public String normalize(String path) throws IOException {
+    return in.normalize(path);
+  }
+
+  @Override
+  protected boolean deleteOldIndexDirectory(String oldDirPath) throws IOException {
+    return in.deleteOldIndexDirectory(oldDirPath);
+  }
+
+  @Override
+  public void initCoreContainer(CoreContainer cc) {
+    in.initCoreContainer(cc);
+  }
+
+  @Override
+  protected Directory getBaseDir(Directory dir) {
+    return in.getBaseDir(dir);
+  }
+
+  @Override
+  public void incRef(Directory dir) {
+    // unwrap
+    if (dir instanceof MetricsDirectory) {
+      dir = ((MetricsDirectory)dir).getDelegate();
+    }
+    in.incRef(dir);
+  }
+
+  @Override
+  public boolean isPersistent() {
+    return in.isPersistent();
+  }
+
+  @Override
+  public void inform(SolrCore core) {
+    if (in instanceof  SolrCoreAware) {
+      ((SolrCoreAware)in).inform(core);
+    }
+  }
+
+  @Override
+  public void release(Directory dir) throws IOException {
+    // unwrap
+    if (dir instanceof MetricsDirectory) {
+      dir = ((MetricsDirectory)dir).getDelegate();
+    }
+    in.release(dir);
+  }
+
+
+
+  private static final String SEGMENTS = "segments";
+  private static final String SEGMENTS_PREFIX = "segments_";
+  private static final String PENDING_SEGMENTS_PREFIX = "pending_segments_";
+  private static final String TEMP = "temp";
+  private static final String OTHER = "other";
+
+  public static class MetricsDirectory extends FilterDirectory {
+
+    private final Directory in;
+    private final String registry;
+    private final SolrMetricManager metricManager;
+    private final Meter totalReads;
+    private final Histogram totalReadSizes;
+    private final Meter totalWrites;
+    private final Histogram totalWriteSizes;
+    private final boolean directoryDetails;
+
+    private final String PREFIX = SolrInfoMBean.Category.DIRECTORY.toString() + ".";
+
+    public MetricsDirectory(SolrMetricManager metricManager, String registry, Directory in, boolean directoryDetails) throws IOException {
+      super(in);
+      this.metricManager = metricManager;
+      this.registry = registry;
+      this.in = in;
+      this.directoryDetails = directoryDetails;
+      this.totalReads = metricManager.meter(registry, "reads", SolrInfoMBean.Category.DIRECTORY.toString(), "total");
+      this.totalWrites = metricManager.meter(registry, "writes", SolrInfoMBean.Category.DIRECTORY.toString(), "total");
+      if (directoryDetails) {
+        this.totalReadSizes = metricManager.histogram(registry, "readSizes", SolrInfoMBean.Category.DIRECTORY.toString(), "total");
+        this.totalWriteSizes = metricManager.histogram(registry, "writeSizes", SolrInfoMBean.Category.DIRECTORY.toString(), "total");
+      } else {
+        this.totalReadSizes = null;
+        this.totalWriteSizes = null;
+      }
+    }
+
+    private String getMetricName(String name, boolean output) {
+      if (!directoryDetails) {
+        return null;
+      }
+      String lastName;
+      if (name.startsWith(SEGMENTS_PREFIX) || name.startsWith(PENDING_SEGMENTS_PREFIX)) {
+        lastName = SEGMENTS;
+      } else {
+        int pos = name.lastIndexOf('.');
+        if (pos != -1 && name.length() > pos + 1) {
+          lastName = name.substring(pos + 1);
+        } else {
+          lastName = OTHER;
+        }
+      }
+      StringBuilder sb = new StringBuilder(PREFIX);
+      sb.append(lastName);
+      sb.append('.');
+      if (output) {
+        sb.append("write");
+      } else {
+        sb.append("read");
+      }
+      return sb.toString();
+    }
+
+    @Override
+    public IndexOutput createOutput(String name, IOContext context) throws IOException {
+      IndexOutput output = in.createOutput(name, context);
+      if (output != null) {
+        return new MetricsOutput(totalWrites, totalWriteSizes, metricManager, registry, getMetricName(name, true), output);
+      } else {
+        return null;
+      }
+    }
+
+    @Override
+    public IndexOutput createTempOutput(String prefix, String suffix, IOContext context) throws IOException {
+      IndexOutput output = in.createTempOutput(prefix, suffix, context);
+      if (output != null) {
+        return new MetricsOutput(totalWrites, totalWriteSizes, metricManager, registry, getMetricName(TEMP, true), output);
+      } else {
+        return null;
+      }
+    }
+
+    @Override
+    public IndexInput openInput(String name, IOContext context) throws IOException {
+      IndexInput input = in.openInput(name, context);
+      if (input != null) {
+        return new MetricsInput(totalReads, totalReadSizes, metricManager, registry, getMetricName(name, false), input);
+      } else {
+        return null;
+      }
+    }
+  }
+
+  public static class MetricsOutput extends IndexOutput {
+    private final IndexOutput in;
+    private final Histogram histogram;
+    private final Meter meter;
+    private final Meter totalMeter;
+    private final Histogram totalHistogram;
+    private final boolean withDetails;
+
+    public MetricsOutput(Meter totalMeter, Histogram totalHistogram, SolrMetricManager metricManager,
+                         String registry, String metricName, IndexOutput in) {
+      super(in.toString(), in.getName());
+      this.in = in;
+      this.totalMeter = totalMeter;
+      this.totalHistogram = totalHistogram;
+      if (metricName != null && totalHistogram != null) {
+        withDetails = true;
+        String histName = metricName + "Sizes";
+        String meterName = metricName + "s";
+        this.histogram = metricManager.histogram(registry, histName);
+        this.meter = metricManager.meter(registry, meterName);
+      } else {
+        withDetails = false;
+        this.histogram = null;
+        this.meter = null;
+      }
+    }
+
+    @Override
+    public void writeByte(byte b) throws IOException {
+      in.writeByte(b);
+      totalMeter.mark();
+      if (withDetails) {
+        totalHistogram.update(1);
+        meter.mark();
+        histogram.update(1);
+      }
+    }
+
+    @Override
+    public void writeBytes(byte[] b, int offset, int length) throws IOException {
+      in.writeBytes(b, offset, length);
+      totalMeter.mark(length);
+      if (withDetails) {
+        totalHistogram.update(length);
+        meter.mark(length);
+        histogram.update(length);
+      }
+    }
+
+    @Override
+    public void close() throws IOException {
+      in.close();
+    }
+
+    @Override
+    public long getFilePointer() {
+      return in.getFilePointer();
+    }
+
+    @Override
+    public long getChecksum() throws IOException {
+      return in.getChecksum();
+    }
+  }
+
+  public static class MetricsInput extends IndexInput {
+    private final IndexInput in;
+    private final Meter totalMeter;
+    private final Histogram totalHistogram;
+    private final Histogram histogram;
+    private final Meter meter;
+    private final boolean withDetails;
+
+    public MetricsInput(Meter totalMeter, Histogram totalHistogram, SolrMetricManager metricManager, String registry, String metricName, IndexInput in) {
+      super(in.toString());
+      this.in = in;
+      this.totalMeter = totalMeter;
+      this.totalHistogram = totalHistogram;
+      if (metricName != null && totalHistogram != null) {
+        withDetails = true;
+        String histName = metricName + "Sizes";
+        String meterName = metricName + "s";
+        this.histogram = metricManager.histogram(registry, histName);
+        this.meter = metricManager.meter(registry, meterName);
+      } else {
+        withDetails = false;
+        this.histogram = null;
+        this.meter = null;
+      }
+    }
+
+    public MetricsInput(Meter totalMeter, Histogram totalHistogram, Histogram histogram, Meter meter, IndexInput in) {
+      super(in.toString());
+      this.in = in;
+      this.totalMeter = totalMeter;
+      this.totalHistogram  = totalHistogram;
+      this.histogram = histogram;
+      this.meter = meter;
+      if (totalHistogram != null && meter != null && histogram != null) {
+        withDetails = true;
+      } else {
+        withDetails = false;
+      }
+    }
+
+    @Override
+    public void close() throws IOException {
+      in.close();
+    }
+
+    @Override
+    public long getFilePointer() {
+      return in.getFilePointer();
+    }
+
+    @Override
+    public void seek(long pos) throws IOException {
+      in.seek(pos);
+    }
+
+    @Override
+    public long length() {
+      return in.length();
+    }
+
+    @Override
+    public IndexInput clone() {
+      return new MetricsInput(totalMeter, totalHistogram, histogram, meter, in.clone());
+    }
+
+    @Override
+    public IndexInput slice(String sliceDescription, long offset, long length) throws IOException {
+      IndexInput slice = in.slice(sliceDescription, offset, length);
+      if (slice != null) {
+        return new MetricsInput(totalMeter, totalHistogram, histogram, meter, slice);
+      } else {
+        return null;
+      }
+    }
+
+    @Override
+    public byte readByte() throws IOException {
+      totalMeter.mark();
+      if (withDetails) {
+        totalHistogram.update(1);
+        meter.mark();
+        histogram.update(1);
+      }
+      return in.readByte();
+    }
+
+    @Override
+    public void readBytes(byte[] b, int offset, int len) throws IOException {
+      totalMeter.mark(len);
+      if (withDetails) {
+        totalHistogram.update(len);
+        meter.mark(len);
+        histogram.update(len);
+      }
+      in.readBytes(b, offset, len);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/48ca9fc3/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 a8d7738..a9fec5a 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -642,7 +642,14 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
       dirFactory = new NRTCachingDirectoryFactory();
       dirFactory.initCoreContainer(getCoreDescriptor().getCoreContainer());
     }
-    return dirFactory;
+    if (solrConfig.indexConfig.metricsInfo != null && solrConfig.indexConfig.metricsInfo.isEnabled()) {
+      final DirectoryFactory factory = new MetricsDirectoryFactory(coreDescriptor.getCoreContainer().getMetricManager(),
+          coreMetricManager.getRegistryName(), dirFactory);
+        factory.init(solrConfig.indexConfig.metricsInfo.initArgs);
+      return factory;
+    } else {
+      return dirFactory;
+    }
   }
 
   private void initIndexReaderFactory() {
@@ -846,6 +853,8 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
     resourceLoader = config.getResourceLoader();
     this.solrConfig = config;
     this.configSetProperties = configSetProperties;
+    // Initialize the metrics manager
+    this.coreMetricManager = initCoreMetricManager(config);
 
     if (updateHandler == null) {
       directoryFactory = initDirectoryFactory();
@@ -863,17 +872,14 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
 
     checkVersionFieldExistsInSchema(schema, coreDescriptor);
 
-    // Initialize the metrics manager
-    this.coreMetricManager = initCoreMetricManager(config);
-
     SolrMetricManager metricManager = this.coreDescriptor.getCoreContainer().getMetricManager();
 
     // initialize searcher-related metrics
-    newSearcherCounter = metricManager.counter(coreMetricManager.getRegistryName(), "newSearcher");
-    newSearcherTimer = metricManager.timer(coreMetricManager.getRegistryName(), "newSearcherTime");
-    newSearcherWarmupTimer = metricManager.timer(coreMetricManager.getRegistryName(), "newSearcherWarmup");
-    newSearcherMaxReachedCounter = metricManager.counter(coreMetricManager.getRegistryName(), "newSearcherMaxReached");
-    newSearcherOtherErrorsCounter = metricManager.counter(coreMetricManager.getRegistryName(), "newSearcherErrors");
+    newSearcherCounter = metricManager.counter(coreMetricManager.getRegistryName(), "new", Category.SEARCHER.toString());
+    newSearcherTimer = metricManager.timer(coreMetricManager.getRegistryName(), "time", Category.SEARCHER.toString(), "new");
+    newSearcherWarmupTimer = metricManager.timer(coreMetricManager.getRegistryName(), "warmup", Category.SEARCHER.toString(), "new");
+    newSearcherMaxReachedCounter = metricManager.counter(coreMetricManager.getRegistryName(), "maxReached", Category.SEARCHER.toString(), "new");
+    newSearcherOtherErrorsCounter = metricManager.counter(coreMetricManager.getRegistryName(), "errors", Category.SEARCHER.toString(), "new");
 
     // Initialize JMX
     this.infoRegistry = initInfoRegistry(name, config);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/48ca9fc3/solr/core/src/java/org/apache/solr/core/SolrDeletionPolicy.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrDeletionPolicy.java b/solr/core/src/java/org/apache/solr/core/SolrDeletionPolicy.java
index 34482cd..eba2964 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrDeletionPolicy.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrDeletionPolicy.java
@@ -114,6 +114,9 @@ public class SolrDeletionPolicy extends IndexDeletionPolicy implements NamedList
 
     protected void appendDetails(StringBuilder sb, IndexCommit c) {
       Directory dir = c.getDirectory();
+      if (dir instanceof MetricsDirectoryFactory.MetricsDirectory) { // unwrap
+        dir = ((MetricsDirectoryFactory.MetricsDirectory) dir).getDelegate();
+      }
       if (dir instanceof FSDirectory) {
         FSDirectory fsd = (FSDirectory) dir;
         sb.append("dir=").append(fsd.getDirectory());
@@ -194,6 +197,9 @@ public class SolrDeletionPolicy extends IndexDeletionPolicy implements NamedList
   private String getId(IndexCommit commit) {
     StringBuilder sb = new StringBuilder();
     Directory dir = commit.getDirectory();
+    if (dir instanceof MetricsDirectoryFactory.MetricsDirectory) { // unwrap
+      dir = ((MetricsDirectoryFactory.MetricsDirectory) dir).getDelegate();
+    }
 
     // For anything persistent, make something that will
     // be the same, regardless of the Directory instance.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/48ca9fc3/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 c5fb84b..c64af47 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java
@@ -32,7 +32,7 @@ public interface SolrInfoMBean {
   /**
    * Category of {@link SolrCore} component.
    */
-  enum Category { CORE, QUERYHANDLER, UPDATEHANDLER, CACHE, HIGHLIGHTING, QUERYPARSER, OTHER }
+  enum Category { CORE, QUERYHANDLER, UPDATEHANDLER, CACHE, HIGHLIGHTING, QUERYPARSER, SEARCHER, INDEX, DIRECTORY, OTHER }
 
   /**
    * Top-level group of beans for a subsystem.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/48ca9fc3/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java b/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java
index 1d8793a..37c15ed 100644
--- a/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java
@@ -142,7 +142,7 @@ public class StandardDirectoryFactory extends CachingDirectoryFactory {
 
     super.move(fromDir, toDir, fileName, ioContext);
   }
-  
+
   // perform an atomic rename if possible
   public void renameWithOverwrite(Directory dir, String fileName, String toName) throws IOException {
     Directory baseDir = getBaseDir(dir);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/48ca9fc3/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java b/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java
index 1a9801f..d484e85 100644
--- a/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java
+++ b/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java
@@ -18,6 +18,7 @@ package org.apache.solr.update;
 
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
@@ -78,6 +79,7 @@ public class SolrIndexConfig implements MapSerializable {
   public final PluginInfo mergePolicyInfo;
   public final PluginInfo mergePolicyFactoryInfo;
   public final PluginInfo mergeSchedulerInfo;
+  public final PluginInfo metricsInfo;
   
   public final PluginInfo mergedSegmentWarmerInfo;
   
@@ -99,6 +101,8 @@ public class SolrIndexConfig implements MapSerializable {
     mergePolicyFactoryInfo = null;
     mergeSchedulerInfo = null;
     mergedSegmentWarmerInfo = null;
+    // enable coarse-grained metrics by default
+    metricsInfo = new PluginInfo("metrics", Collections.emptyMap(), null, null);
   }
   
   /**
@@ -144,6 +148,12 @@ public class SolrIndexConfig implements MapSerializable {
     writeLockTimeout=solrConfig.getInt(prefix+"/writeLockTimeout", def.writeLockTimeout);
     lockType=solrConfig.get(prefix+"/lockType", def.lockType);
 
+    List<PluginInfo> infos = solrConfig.readPluginInfos(prefix + "/metrics", false, false);
+    if (infos.isEmpty()) {
+      metricsInfo = def.metricsInfo;
+    } else {
+      metricsInfo = infos.get(0);
+    }
     mergeSchedulerInfo = getPluginInfo(prefix + "/mergeScheduler", solrConfig, def.mergeSchedulerInfo);
     mergePolicyInfo = getPluginInfo(prefix + "/mergePolicy", solrConfig, def.mergePolicyInfo);
     mergePolicyFactoryInfo = getPluginInfo(prefix + "/mergePolicyFactory", solrConfig, def.mergePolicyFactoryInfo);
@@ -197,6 +207,9 @@ public class SolrIndexConfig implements MapSerializable {
         "lockType", lockType,
         "infoStreamEnabled", infoStream != InfoStream.NO_OUTPUT);
     if(mergeSchedulerInfo != null) m.put("mergeScheduler",mergeSchedulerInfo);
+    if (metricsInfo != null) {
+      m.put("metrics", metricsInfo);
+    }
     if (mergePolicyInfo != null) {
       m.put("mergePolicy", mergePolicyInfo);
     } else if (mergePolicyFactoryInfo != null) {
@@ -237,7 +250,8 @@ public class SolrIndexConfig implements MapSerializable {
     iwc.setSimilarity(schema.getSimilarity());
     MergePolicy mergePolicy = buildMergePolicy(schema);
     iwc.setMergePolicy(mergePolicy);
-    iwc.setMergeScheduler(buildMergeScheduler(schema));
+    MergeScheduler mergeScheduler = buildMergeScheduler(schema);
+    iwc.setMergeScheduler(mergeScheduler);
     iwc.setInfoStream(infoStream);
 
     if (mergePolicy instanceof SortingMergePolicy) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/48ca9fc3/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java b/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java
index d75214a..626bc8e 100644
--- a/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java
+++ b/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java
@@ -20,12 +20,19 @@ import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Meter;
+import com.codahale.metrics.Timer;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.index.IndexDeletionPolicy;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.MergePolicy;
+import org.apache.lucene.index.SegmentCommitInfo;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.InfoStream;
 import org.apache.solr.common.util.IOUtils;
@@ -33,6 +40,8 @@ import org.apache.solr.common.util.SuppressForbidden;
 import org.apache.solr.core.DirectoryFactory;
 import org.apache.solr.core.DirectoryFactory.DirContext;
 import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.SolrInfoMBean;
+import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.schema.IndexSchema;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -60,6 +69,28 @@ public class SolrIndexWriter extends IndexWriter {
   private InfoStream infoStream;
   private Directory directory;
 
+  // metrics
+  private long majorMergeDocs = 512 * 1024;
+  private final Timer majorMerge;
+  private final Timer minorMerge;
+  private final Meter majorMergedDocs;
+  private final Meter majorDeletedDocs;
+  private final Counter mergeErrors;
+  private final Meter flushMeter; // original counter is package-private in IndexWriter
+  private final boolean mergeDetails;
+  private final AtomicInteger runningMajorMerges = new AtomicInteger();
+  private final Gauge<Integer> runningMajorMergesGauge;
+  private final AtomicInteger runningMinorMerges = new AtomicInteger();
+  private final Gauge<Integer> runningMinorMergesGauge;
+  private final AtomicInteger runningMajorMergesSegments = new AtomicInteger();
+  private final Gauge<Integer> runningMajorMergesSegmentsGauge;
+  private final AtomicInteger runningMinorMergesSegments = new AtomicInteger();
+  private final Gauge<Integer> runningMinorMergesSegmentsGauge;
+  private final AtomicLong runningMajorMergesDocs = new AtomicLong();
+  private final Gauge<Long> runningMajorMergesDocsGauge;
+  private final AtomicLong runningMinorMergesDocs = new AtomicLong();
+  private final Gauge<Long> runningMinorMergesDocsGauge;
+
   public static SolrIndexWriter create(SolrCore core, String name, String path, DirectoryFactory directoryFactory, boolean create, IndexSchema schema, SolrIndexConfig config, IndexDeletionPolicy delPolicy, Codec codec) throws IOException {
 
     SolrIndexWriter w = null;
@@ -84,6 +115,20 @@ public class SolrIndexWriter extends IndexWriter {
     this.directory = d;
     numOpens.incrementAndGet();
     log.debug("Opened Writer " + name);
+    // no metrics
+    minorMerge = null;
+    majorMerge = null;
+    mergeErrors = null;
+    majorMergedDocs = null;
+    majorDeletedDocs = null;
+    runningMinorMergesGauge = null;
+    runningMinorMergesDocsGauge = null;
+    runningMinorMergesSegmentsGauge = null;
+    runningMajorMergesGauge = null;
+    runningMajorMergesDocsGauge = null;
+    runningMajorMergesSegmentsGauge = null;
+    flushMeter = null;
+    mergeDetails = false;
   }
 
   private SolrIndexWriter(SolrCore core, String name, String path, Directory directory, boolean create, IndexSchema schema, SolrIndexConfig config, IndexDeletionPolicy delPolicy, Codec codec) throws IOException {
@@ -97,6 +142,51 @@ public class SolrIndexWriter extends IndexWriter {
     infoStream = getConfig().getInfoStream();
     this.directory = directory;
     numOpens.incrementAndGet();
+    SolrMetricManager metricManager = core.getCoreDescriptor().getCoreContainer().getMetricManager();
+    String registry = core.getCoreMetricManager().getRegistryName();
+    minorMerge = metricManager.timer(registry, "minor", SolrInfoMBean.Category.INDEX.toString(), "merge");
+    majorMerge = metricManager.timer(registry, "major", SolrInfoMBean.Category.INDEX.toString(), "merge");
+    mergeErrors = metricManager.counter(registry, "errors", SolrInfoMBean.Category.INDEX.toString(), "merge");
+    runningMajorMergesGauge = () -> runningMajorMerges.get();
+    runningMinorMergesGauge = () -> runningMinorMerges.get();
+    runningMajorMergesDocsGauge = () -> runningMajorMergesDocs.get();
+    runningMinorMergesDocsGauge = () -> runningMinorMergesDocs.get();
+    runningMajorMergesSegmentsGauge = () -> runningMajorMergesSegments.get();
+    runningMinorMergesSegmentsGauge = () -> runningMinorMergesSegments.get();
+    metricManager.register(registry, runningMajorMergesGauge, true, "running", SolrInfoMBean.Category.INDEX.toString(), "merge", "major");
+    metricManager.register(registry, runningMinorMergesGauge, true, "running", SolrInfoMBean.Category.INDEX.toString(), "merge", "minor");
+    metricManager.register(registry, runningMajorMergesDocsGauge, true, "running.docs", SolrInfoMBean.Category.INDEX.toString(), "merge", "major");
+    metricManager.register(registry, runningMinorMergesDocsGauge, true, "running.docs", SolrInfoMBean.Category.INDEX.toString(), "merge", "minor");
+    metricManager.register(registry, runningMajorMergesSegmentsGauge, true, "running.segments", SolrInfoMBean.Category.INDEX.toString(), "merge", "major");
+    metricManager.register(registry, runningMinorMergesSegmentsGauge, true, "running.segments", SolrInfoMBean.Category.INDEX.toString(), "merge", "minor");
+    flushMeter = metricManager.meter(registry, "flush", SolrInfoMBean.Category.INDEX.toString());
+    if (config.metricsInfo != null && config.metricsInfo.initArgs != null) {
+      Object v = config.metricsInfo.initArgs.get("majorMergeDocs");
+      if (v != null) {
+        try {
+          majorMergeDocs = Long.parseLong(String.valueOf(v));
+        } catch (Exception e) {
+          log.warn("Invalid 'majorMergeDocs' argument, using default 512k", e);
+        }
+      }
+      Boolean Details = config.metricsInfo.initArgs.getBooleanArg("mergeDetails");
+      if (Details != null) {
+        mergeDetails = Details;
+      } else {
+        mergeDetails = false;
+      }
+      if (mergeDetails) {
+        majorMergedDocs = metricManager.meter(registry, "docs", SolrInfoMBean.Category.INDEX.toString(), "merge", "major");
+        majorDeletedDocs = metricManager.meter(registry, "deletedDocs", SolrInfoMBean.Category.INDEX.toString(), "merge", "major");
+      } else {
+        majorMergedDocs = null;
+        majorDeletedDocs = null;
+      }
+    } else {
+      mergeDetails = false;
+      majorMergedDocs = null;
+      majorDeletedDocs = null;
+    }
   }
 
   @SuppressForbidden(reason = "Need currentTimeMillis, commit time should be used only for debugging purposes, " +
@@ -112,6 +202,60 @@ public class SolrIndexWriter extends IndexWriter {
     this.directoryFactory = factory;
   }
 
+  // we override this method to collect metrics for merges.
+  @Override
+  public void merge(MergePolicy.OneMerge merge) throws IOException {
+    long deletedDocs = 0;
+    long totalNumDocs = merge.totalNumDocs();
+    for (SegmentCommitInfo info : merge.segments) {
+      totalNumDocs -= info.getDelCount();
+      deletedDocs += info.getDelCount();
+    }
+    boolean major = totalNumDocs > majorMergeDocs;
+    int segmentsCount = merge.segments.size();
+    Timer.Context context;
+    if (major) {
+      runningMajorMerges.incrementAndGet();
+      runningMajorMergesDocs.addAndGet(totalNumDocs);
+      runningMajorMergesSegments.addAndGet(segmentsCount);
+      if (mergeDetails) {
+        majorMergedDocs.mark(totalNumDocs);
+        majorDeletedDocs.mark(deletedDocs);
+      }
+      context = majorMerge.time();
+    } else {
+      runningMinorMerges.incrementAndGet();
+      runningMinorMergesDocs.addAndGet(totalNumDocs);
+      runningMinorMergesSegments.addAndGet(segmentsCount);
+      context = minorMerge.time();
+    }
+    try {
+      super.merge(merge);
+    } catch (Throwable t) {
+      mergeErrors.inc();
+      throw t;
+    } finally {
+      context.stop();
+      if (major) {
+        runningMajorMerges.decrementAndGet();
+        runningMajorMergesDocs.addAndGet(-totalNumDocs);
+        runningMajorMergesSegments.addAndGet(-segmentsCount);
+      } else {
+        runningMinorMerges.decrementAndGet();
+        runningMinorMergesDocs.addAndGet(-totalNumDocs);
+        runningMinorMergesSegments.addAndGet(-segmentsCount);
+      }
+    }
+  }
+
+  @Override
+  protected void doAfterFlush() throws IOException {
+    if (flushMeter != null) { // this is null when writer is used only for snapshot cleanup
+      flushMeter.mark();
+    }
+    super.doAfterFlush();
+  }
+
   /**
    * use DocumentBuilder now...
    * private final void addField(Document doc, String name, String val) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/48ca9fc3/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 af5a0b5..4a83c86 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
@@ -40,7 +40,7 @@ import org.apache.solr.common.util.NamedList;
 public class MetricUtils {
 
   /**
-   * Adds metrics from a Timer to a NamedList, using well-known names.
+   * Adds metrics from a Timer to a NamedList, using well-known back-compat names.
    * @param lst The NamedList to add the metrics data to
    * @param timer The Timer to extract the metrics from
    */
@@ -68,7 +68,7 @@ public class MetricUtils {
   }
 
   /**
-   * Returns a NamedList respresentation of the given metric registry. Only those metrics
+   * Returns a NamedList representation of the given metric registry. Only those metrics
    * are converted to NamedList which match at least one of the given MetricFilter instances.
    *
    * @param registry      the {@link MetricRegistry} to be converted to NamedList
@@ -104,31 +104,74 @@ public class MetricUtils {
   static NamedList histogramToNamedList(Histogram histogram) {
     NamedList response = new NamedList();
     Snapshot snapshot = histogram.getSnapshot();
-    response.add("requests", histogram.getCount());
-    response.add("minTime", nsToMs(snapshot.getMin()));
-    response.add("maxTime", nsToMs(snapshot.getMax()));
-    response.add("avgTimePerRequest", nsToMs(snapshot.getMean()));
-    response.add("medianRequestTime", nsToMs(snapshot.getMedian()));
-    response.add("75thPcRequestTime", nsToMs(snapshot.get75thPercentile()));
-    response.add("95thPcRequestTime", nsToMs(snapshot.get95thPercentile()));
-    response.add("99thPcRequestTime", nsToMs(snapshot.get99thPercentile()));
-    response.add("999thPcRequestTime", nsToMs(snapshot.get999thPercentile()));
+    response.add("count", histogram.getCount());
+    // non-time based values
+    addSnapshot(response, snapshot, false);
     return response;
   }
 
+  // optionally convert ns to ms
+  static double nsToMs(boolean convert, double value) {
+    if (convert) {
+      return nsToMs(value);
+    } else {
+      return value;
+    }
+  }
+
+  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();
-    addMetrics(response, timer);
+    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);
     return response;
   }
 
   static NamedList meterToNamedList(Meter meter) {
     NamedList response = new NamedList();
-    response.add("requests", meter.getCount());
-    response.add("avgRequestsPerSecond", meter.getMeanRate());
-    response.add("1minRateRequestsPerSecond", meter.getOneMinuteRate());
-    response.add("5minRateRequestsPerSecond", meter.getFiveMinuteRate());
-    response.add("15minRateRequestsPerSecond", meter.getFifteenMinuteRate());
+    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());
     return response;
   }
 
@@ -140,7 +183,7 @@ public class MetricUtils {
 
   static NamedList counterToNamedList(Counter counter) {
     NamedList response = new NamedList();
-    response.add("requests", counter.getCount());
+    response.add("count", counter.getCount());
     return response;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/48ca9fc3/solr/core/src/test-files/solr/collection1/conf/solrconfig-indexmetrics.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-indexmetrics.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-indexmetrics.xml
new file mode 100644
index 0000000..1acf18d
--- /dev/null
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-indexmetrics.xml
@@ -0,0 +1,57 @@
+<?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>
+
+  <dataDir>${solr.data.dir:}</dataDir>
+
+  <directoryFactory name="DirectoryFactory"
+                    class="${solr.directoryFactory:solr.NRTCachingDirectoryFactory}"/>
+  <schemaFactory class="ClassicIndexSchemaFactory"/>
+
+  <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
+
+  <indexConfig>
+    <metrics>
+      <bool name="directoryDetails">${solr.tests.directoryDetails:false}</bool>
+      <bool name="mergeDetails">${solr.tests.mergeDetails:false}</bool>
+    </metrics>
+    <!-- intentionally set very low values here to trigger multiple flushes and merges.
+         DO NOT USE THESE ABSURD VALUES IN PRODUCTION. -->
+    <mergeFactor>3</mergeFactor>
+    <maxBufferedDocs>100</maxBufferedDocs>
+
+    <mergePolicy class="org.apache.lucene.index.TieredMergePolicy"/>
+  </indexConfig>
+
+  <updateHandler class="solr.DirectUpdateHandler2">
+    <commitWithin>
+      <softCommit>${solr.commitwithin.softcommit:true}</softCommit>
+    </commitWithin>
+
+  </updateHandler>
+  <requestHandler name="/select" class="solr.SearchHandler">
+    <lst name="defaults">
+      <str name="echoParams">explicit</str>
+      <str name="indent">true</str>
+      <str name="df">text</str>
+    </lst>
+
+  </requestHandler>
+</config>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/48ca9fc3/solr/core/src/test/org/apache/solr/core/TestConfig.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/TestConfig.java b/solr/core/src/test/org/apache/solr/core/TestConfig.java
index 55e1e17..8244b32 100644
--- a/solr/core/src/test/org/apache/solr/core/TestConfig.java
+++ b/solr/core/src/test/org/apache/solr/core/TestConfig.java
@@ -128,6 +128,8 @@ public class TestConfig extends SolrTestCaseJ4 {
 
     ++numDefaultsTested; assertEquals("default infoStream", InfoStream.NO_OUTPUT, sic.infoStream);
 
+    ++numDefaultsTested; assertNotNull("default metrics", sic.metricsInfo);
+
     // mergePolicyInfo and mergePolicyFactoryInfo are mutually exclusive
     // so ++ count them only once for both instead of individually
     ++numDefaultsTested; ++numNullDefaults;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/48ca9fc3/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java b/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
index 685ef99..345b86d 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
@@ -66,6 +66,8 @@ import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.CachingDirectoryFactory;
 import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.DirectoryFactory;
+import org.apache.solr.core.MetricsDirectoryFactory;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.StandardDirectoryFactory;
 import org.apache.solr.core.snapshots.SolrSnapshotMetaDataManager;
@@ -895,12 +897,21 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
     }
   }
 
+  private CachingDirectoryFactory getCachingDirectoryFactory(SolrCore core) {
+    DirectoryFactory df = core.getDirectoryFactory();
+    if (df instanceof MetricsDirectoryFactory) {
+      return (CachingDirectoryFactory)((MetricsDirectoryFactory)df).getDelegate();
+    } else {
+      return (CachingDirectoryFactory)df;
+    }
+  }
+
   private void checkForSingleIndex(JettySolrRunner jetty) {
     CoreContainer cores = jetty.getCoreContainer();
     Collection<SolrCore> theCores = cores.getCores();
     for (SolrCore core : theCores) {
       String ddir = core.getDataDir();
-      CachingDirectoryFactory dirFactory = (CachingDirectoryFactory) core.getDirectoryFactory();
+      CachingDirectoryFactory dirFactory = getCachingDirectoryFactory(core);
       synchronized (dirFactory) {
         Set<String> livePaths = dirFactory.getLivePaths();
         // one for data, one for hte index under data and one for the snapshot metadata.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/48ca9fc3/solr/core/src/test/org/apache/solr/handler/admin/CoreMergeIndexesAdminHandlerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/CoreMergeIndexesAdminHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/CoreMergeIndexesAdminHandlerTest.java
index 6f1a802..937cc86 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/CoreMergeIndexesAdminHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/CoreMergeIndexesAdminHandlerTest.java
@@ -24,6 +24,8 @@ import org.apache.lucene.store.LockFactory;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.DirectoryFactory;
+import org.apache.solr.core.MetricsDirectoryFactory;
 import org.apache.solr.core.MockFSDirectoryFactory;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.response.SolrQueryResponse;
@@ -75,7 +77,13 @@ public class CoreMergeIndexesAdminHandlerTest extends SolrTestCaseJ4 {
     final CoreAdminHandler admin = new CoreAdminHandler(cores);
 
     try (SolrCore core = cores.getCore("collection1")) {
-      FailingDirectoryFactory dirFactory = (FailingDirectoryFactory)core.getDirectoryFactory();
+      DirectoryFactory df = core.getDirectoryFactory();
+      FailingDirectoryFactory dirFactory;
+      if (df instanceof MetricsDirectoryFactory) {
+        dirFactory = (FailingDirectoryFactory)((MetricsDirectoryFactory)df).getDelegate();
+      } else {
+        dirFactory = (FailingDirectoryFactory)df;
+      }
 
       try {
         dirFactory.fail = true;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/48ca9fc3/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 3667285..e15778d 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
@@ -49,9 +49,9 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 {
     assertNotNull(values.get("solr.node"));
     NamedList nl = (NamedList) values.get("solr.core.collection1");
     assertNotNull(nl);
-    assertNotNull(nl.get("newSearcherErrors")); // counter type
-    assertNotNull(((NamedList) nl.get("newSearcherErrors")).get("requests"));
-    assertEquals(0L, ((NamedList) nl.get("newSearcherErrors")).get("requests"));
+    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"));
     nl = (NamedList) values.get("solr.node");
     assertNotNull(nl.get("cores.loaded")); // int gauge
     assertEquals(1, ((NamedList) nl.get("cores.loaded")).get("value"));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/48ca9fc3/solr/core/src/test/org/apache/solr/update/SolrIndexConfigTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/SolrIndexConfigTest.java b/solr/core/src/test/org/apache/solr/update/SolrIndexConfigTest.java
index 4fec5c3..7d1c4c7 100644
--- a/solr/core/src/test/org/apache/solr/update/SolrIndexConfigTest.java
+++ b/solr/core/src/test/org/apache/solr/update/SolrIndexConfigTest.java
@@ -197,6 +197,7 @@ public class SolrIndexConfigTest extends SolrTestCaseJ4 {
     } else {
       assertNull(m.get("mergedSegmentWarmer"));
     }
+    ++mSizeExpected; assertNotNull(m.get("metrics"));
 
     assertEquals(mSizeExpected, m.size());
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/48ca9fc3/solr/core/src/test/org/apache/solr/update/SolrIndexMetricsTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/SolrIndexMetricsTest.java b/solr/core/src/test/org/apache/solr/update/SolrIndexMetricsTest.java
new file mode 100644
index 0000000..e17b1bd
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/update/SolrIndexMetricsTest.java
@@ -0,0 +1,94 @@
+/*
+ * 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.update;
+
+import java.util.Map;
+
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.Meter;
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.Timer;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.request.SolrQueryRequest;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test proper registration and collection of index and directory metrics.
+ */
+public class SolrIndexMetricsTest extends SolrTestCaseJ4 {
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    System.setProperty("solr.tests.mergeDetails", "true");
+    System.setProperty("solr.tests.directoryDetails", "true");
+    initCore("solrconfig-indexmetrics.xml", "schema.xml");
+  }
+
+  @Test
+  public void testIndexMetrics() throws Exception {
+    SolrQueryRequest req = lrf.makeRequest();
+    UpdateHandler uh = req.getCore().getUpdateHandler();
+    AddUpdateCommand add = new AddUpdateCommand(req);
+    for (int i = 0; i < 1000; i++) {
+      add.clear();
+      add.solrDoc = new SolrInputDocument();
+      add.solrDoc.addField("id", "" + i);
+      add.solrDoc.addField("foo_s", "foo-" + i);
+      uh.addDoc(add);
+    }
+    uh.commit(new CommitUpdateCommand(req, false));
+    MetricRegistry registry = h.getCoreContainer().getMetricManager().registry(h.getCore().getCoreMetricManager().getRegistryName());
+    assertNotNull(registry);
+    // make sure all merges are finished
+    h.reload();
+
+    Map<String, Metric> metrics = registry.getMetrics();
+
+    assertTrue(metrics.entrySet().stream().filter(e -> e.getKey().startsWith("INDEX")).count() >= 12);
+    // this is variable, depending on the codec and the number of created files
+    assertTrue(metrics.entrySet().stream().filter(e -> e.getKey().startsWith("DIRECTORY")).count() > 50);
+
+    // check basic index meters
+    Timer timer = (Timer)metrics.get("INDEX.merge.minor");
+    assertEquals("minorMerge: " + timer.getCount(), 4, timer.getCount());
+    timer = (Timer)metrics.get("INDEX.merge.major");
+    assertEquals("majorMerge: " + timer.getCount(), 0, timer.getCount());
+    Meter meter = (Meter)metrics.get("INDEX.merge.major.docs");
+    assertEquals("majorMergeDocs: " + meter.getCount(), 0, meter.getCount());
+    meter = (Meter)metrics.get("INDEX.flush");
+    assertEquals("flush: " + meter.getCount(), 19, meter.getCount());
+
+    // check basic directory meters
+    meter = (Meter)metrics.get("DIRECTORY.total.reads");
+    assertTrue("totalReads", meter.getCount() > 0);
+    meter = (Meter)metrics.get("DIRECTORY.total.writes");
+    assertTrue("totalWrites", meter.getCount() > 0);
+    Histogram histogram = (Histogram)metrics.get("DIRECTORY.total.readSizes");
+    assertTrue("readSizes", histogram.getCount() > 0);
+    histogram = (Histogram)metrics.get("DIRECTORY.total.writeSizes");
+    assertTrue("writeSizes", histogram.getCount() > 0);
+    // check detailed meters
+    meter = (Meter)metrics.get("DIRECTORY.segments.writes");
+    assertTrue("segmentsWrites", meter.getCount() > 0);
+    histogram = (Histogram)metrics.get("DIRECTORY.segments.writeSizes");
+    assertTrue("segmentsWriteSizes", histogram.getCount() > 0);
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/48ca9fc3/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 31e8154..e39ad6e 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
@@ -23,7 +23,6 @@ import com.codahale.metrics.Snapshot;
 import com.codahale.metrics.Timer;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.util.NamedList;
-import org.apache.solr.common.util.SimpleOrderedMap;
 import org.junit.Test;
 
 public class MetricUtilsTest extends SolrTestCaseJ4 {
@@ -34,24 +33,23 @@ public class MetricUtilsTest extends SolrTestCaseJ4 {
     final Timer timer = new Timer();
     final int iterations = random().nextInt(100);
     for (int i = 0; i < iterations; ++i) {
-      timer.update(random().nextInt(), TimeUnit.NANOSECONDS);
+      timer.update(Math.abs(random().nextInt()) + 1, TimeUnit.NANOSECONDS);
     }
     // obtain timer metrics
-    final NamedList<Object> lst = new SimpleOrderedMap<>();
-    MetricUtils.addMetrics(lst, timer);
+    NamedList lst = MetricUtils.timerToNamedList(timer);
     // check that expected metrics were obtained
-    assertEquals(lst.size(), 9);
+    assertEquals(14, lst.size());
     final Snapshot snapshot = timer.getSnapshot();
     // cannot test avgRequestsPerMinute directly because mean rate changes as time increases!
     // assertEquals(lst.get("avgRequestsPerSecond"), timer.getMeanRate());
-    assertEquals(lst.get("5minRateRequestsPerSecond"), timer.getFiveMinuteRate());
-    assertEquals(lst.get("15minRateRequestsPerSecond"), timer.getFifteenMinuteRate());
-    assertEquals(lst.get("avgTimePerRequest"), MetricUtils.nsToMs(snapshot.getMean()));
-    assertEquals(lst.get("medianRequestTime"), MetricUtils.nsToMs(snapshot.getMedian()));
-    assertEquals(lst.get("75thPcRequestTime"), MetricUtils.nsToMs(snapshot.get75thPercentile()));
-    assertEquals(lst.get("95thPcRequestTime"), MetricUtils.nsToMs(snapshot.get95thPercentile()));
-    assertEquals(lst.get("99thPcRequestTime"), MetricUtils.nsToMs(snapshot.get99thPercentile()));
-    assertEquals(lst.get("999thPcRequestTime"), MetricUtils.nsToMs(snapshot.get999thPercentile()));
+    assertEquals(timer.getFiveMinuteRate(), lst.get("5minRate"));
+    assertEquals(timer.getFifteenMinuteRate(), lst.get("15minRate"));
+    assertEquals(MetricUtils.nsToMs(snapshot.getMean()), lst.get("mean_ms"));
+    assertEquals(MetricUtils.nsToMs(snapshot.getMedian()), lst.get("median_ms"));
+    assertEquals(MetricUtils.nsToMs(snapshot.get75thPercentile()), lst.get("p75_ms"));
+    assertEquals(MetricUtils.nsToMs(snapshot.get95thPercentile()), lst.get("p95_ms"));
+    assertEquals(MetricUtils.nsToMs(snapshot.get99thPercentile()), lst.get("p99_ms"));
+    assertEquals(MetricUtils.nsToMs(snapshot.get999thPercentile()), lst.get("p999_ms"));
   }
 
 }


[33/50] lucene-solr:jira/solr-8593: SOLR-9495: AIOBE with confusing message for incomplete sort spec in Streaming Expression

Posted by kr...@apache.org.
SOLR-9495: AIOBE with confusing message for incomplete sort spec in Streaming Expression


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

Branch: refs/heads/jira/solr-8593
Commit: 61676188d7f592f697933b6051806c0bc55b406a
Parents: 36a691c
Author: Joel Bernstein <jb...@apache.org>
Authored: Fri Dec 30 19:54:00 2016 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Fri Dec 30 19:55:31 2016 -0500

----------------------------------------------------------------------
 .../solr/client/solrj/io/stream/CloudSolrStream.java      |  4 ++++
 .../solr/client/solrj/io/stream/StreamExpressionTest.java | 10 ++++++++++
 2 files changed, 14 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/61676188/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java
index 0580122..f177585 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java
@@ -332,6 +332,10 @@ public class CloudSolrStream extends TupleStream implements Expressible {
 
       String[] spec = s.trim().split("\\s+"); //This should take into account spaces in the sort spec.
       
+      if (spec.length != 2) {
+        throw new IOException("Invalid sort spec:" + s);
+      }
+
       String fieldName = spec[0].trim();
       String order = spec[1].trim();
       

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/61676188/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 3bfe129..936d42f 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
@@ -162,6 +162,16 @@ public class StreamExpressionTest extends SolrCloudTestCase {
       assertTrue(e.getMessage().contains("fl param expected for search function"));
     }
 
+
+    try {
+      expression = StreamExpressionParser.parse("search(" + COLLECTIONORALIAS + ", q=\"blah\", fl=\"id, a_f\", sort=\"a_f\")");
+      stream = new CloudSolrStream(expression, factory);
+      tuples = getTuples(stream);
+      throw new Exception("Should be an exception here");
+    } catch(Exception e) {
+      assertTrue(e.getMessage().contains("Invalid sort spec"));
+    }
+
   }
 
   @Test


[15/50] lucene-solr:jira/solr-8593: LUCENE-5325: Add LongValuesSource and DoubleValuesSource in core

Posted by kr...@apache.org.
LUCENE-5325: Add LongValuesSource and DoubleValuesSource in 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/3f24fd81
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/3f24fd81
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/3f24fd81

Branch: refs/heads/jira/solr-8593
Commit: 3f24fd81c836982be96b9b60082b53177fffe504
Parents: 87b6c2c
Author: Alan Woodward <ro...@apache.org>
Authored: Wed Dec 28 20:10:47 2016 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Thu Dec 29 12:22:40 2016 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   5 +
 .../org/apache/lucene/search/DoubleValues.java  |  38 +++
 .../lucene/search/DoubleValuesSource.java       | 313 +++++++++++++++++++
 .../org/apache/lucene/search/LongValues.java    |  38 +++
 .../apache/lucene/search/LongValuesSource.java  | 217 +++++++++++++
 .../lucene/search/TestDoubleValuesSource.java   | 158 ++++++++++
 .../lucene/search/TestLongValuesSource.java     | 140 +++++++++
 7 files changed, 909 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3f24fd81/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 35314db..12b615d 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -89,6 +89,11 @@ New features
   http://blog.mikemccandless.com/2012/04/lucenes-tokenstreams-are-actually.html
   for details. (Mike McCandless)
 
+* LUCENE-5325: Added LongValuesSource and DoubleValuesSource, intended as
+  type-safe replacements for ValueSource in the queries module.  These
+  expose per-segment LongValues or DoubleValues iterators, similar to the
+  existing DocValues iterator API. (Alan Woodward, Adrien Grand)
+
 Bug Fixes
 
 * LUCENE-7547: JapaneseTokenizerFactory was failing to close the

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3f24fd81/lucene/core/src/java/org/apache/lucene/search/DoubleValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/DoubleValues.java b/lucene/core/src/java/org/apache/lucene/search/DoubleValues.java
new file mode 100644
index 0000000..4f12390
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/DoubleValues.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search;
+
+import java.io.IOException;
+
+/**
+ * Per-segment, per-document double values, which can be calculated at search-time
+ */
+public abstract class DoubleValues {
+
+  /**
+   * Get the double value for the current document
+   */
+  public abstract double doubleValue() throws IOException;
+
+  /**
+   * Advance this instance to the given document id
+   * @return true if there is a value for this document
+   */
+  public abstract boolean advanceExact(int doc) throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3f24fd81/lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java b/lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java
new file mode 100644
index 0000000..4ac8fc1
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.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.search;
+
+import java.io.IOException;
+import java.util.Objects;
+import java.util.function.DoubleToLongFunction;
+import java.util.function.LongToDoubleFunction;
+
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+
+/**
+ * Base class for producing {@link DoubleValues}
+ *
+ * To obtain a {@link DoubleValues} object for a leaf reader, clients should
+ * call {@link #getValues(LeafReaderContext, DoubleValues)}.
+ *
+ * DoubleValuesSource objects for NumericDocValues fields can be obtained by calling
+ * {@link #fromDoubleField(String)}, {@link #fromFloatField(String)}, {@link #fromIntField(String)}
+ * or {@link #fromLongField(String)}, or from {@link #fromField(String, LongToDoubleFunction)} if
+ * special long-to-double encoding is required.
+ *
+ * Scores may be used as a source for value calculations by wrapping a {@link Scorer} using
+ * {@link #fromScorer(Scorer)} and passing the resulting DoubleValues to {@link #getValues(LeafReaderContext, DoubleValues)}.
+ * The scores can then be accessed using the {@link #SCORES} DoubleValuesSource.
+ */
+public abstract class DoubleValuesSource {
+
+  /**
+   * Returns a {@link DoubleValues} instance for the passed-in LeafReaderContext and scores
+   *
+   * If scores are not needed to calculate the values (ie {@link #needsScores() returns false}, callers
+   * may safely pass {@code null} for the {@code scores} parameter.
+   */
+  public abstract DoubleValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException;
+
+  /**
+   * Return true if document scores are needed to calculate values
+   */
+  public abstract boolean needsScores();
+
+  /**
+   * Create a sort field based on the value of this producer
+   * @param reverse true if the sort should be decreasing
+   */
+  public SortField getSortField(boolean reverse) {
+    return new DoubleValuesSortField(this, reverse);
+  }
+
+  /**
+   * Convert to a LongValuesSource by casting the double values to longs
+   */
+  public final LongValuesSource toLongValuesSource() {
+    return new LongValuesSource() {
+      @Override
+      public LongValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+        DoubleValues in = DoubleValuesSource.this.getValues(ctx, scores);
+        return new LongValues() {
+          @Override
+          public long longValue() throws IOException {
+            return (long) in.doubleValue();
+          }
+
+          @Override
+          public boolean advanceExact(int doc) throws IOException {
+            return in.advanceExact(doc);
+          }
+        };
+      }
+
+      @Override
+      public boolean needsScores() {
+        return DoubleValuesSource.this.needsScores();
+      }
+    };
+  }
+
+  /**
+   * Creates a DoubleValuesSource that wraps a generic NumericDocValues field
+   *
+   * @param field the field to wrap, must have NumericDocValues
+   * @param decoder a function to convert the long-valued doc values to doubles
+   */
+  public static DoubleValuesSource fromField(String field, LongToDoubleFunction decoder) {
+    return new FieldValuesSource(field, decoder);
+  }
+
+  /**
+   * Creates a DoubleValuesSource that wraps a double-valued field
+   */
+  public static DoubleValuesSource fromDoubleField(String field) {
+    return fromField(field, Double::longBitsToDouble);
+  }
+
+  /**
+   * Creates a DoubleValuesSource that wraps a float-valued field
+   */
+  public static DoubleValuesSource fromFloatField(String field) {
+    return fromField(field, (v) -> (double)Float.intBitsToFloat((int)v));
+  }
+
+  /**
+   * Creates a DoubleValuesSource that wraps a long-valued field
+   */
+  public static DoubleValuesSource fromLongField(String field) {
+    return fromField(field, (v) -> (double) v);
+  }
+
+  /**
+   * Creates a DoubleValuesSource that wraps an int-valued field
+   */
+  public static DoubleValuesSource fromIntField(String field) {
+    return fromLongField(field);
+  }
+
+  /**
+   * A DoubleValuesSource that exposes a document's score
+   *
+   * If this source is used as part of a values calculation, then callers must not
+   * pass {@code null} as the {@link DoubleValues} parameter on {@link #getValues(LeafReaderContext, DoubleValues)}
+   */
+  public static final DoubleValuesSource SCORES = new DoubleValuesSource() {
+    @Override
+    public DoubleValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+      assert scores != null;
+      return scores;
+    }
+
+    @Override
+    public boolean needsScores() {
+      return true;
+    }
+  };
+
+  /**
+   * Returns a DoubleValues instance that wraps scores returned by a Scorer
+   */
+  public static DoubleValues fromScorer(Scorer scorer) {
+    return new DoubleValues() {
+      @Override
+      public double doubleValue() throws IOException {
+        return scorer.score();
+      }
+
+      @Override
+      public boolean advanceExact(int doc) throws IOException {
+        assert scorer.docID() == doc;
+        return true;
+      }
+    };
+  }
+
+  private static class FieldValuesSource extends DoubleValuesSource {
+
+    final String field;
+    final LongToDoubleFunction decoder;
+
+    private FieldValuesSource(String field, LongToDoubleFunction decoder) {
+      this.field = field;
+      this.decoder = decoder;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+      FieldValuesSource that = (FieldValuesSource) o;
+      return Objects.equals(field, that.field) &&
+          Objects.equals(decoder, that.decoder);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(field, decoder);
+    }
+
+    @Override
+    public DoubleValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+      final NumericDocValues values = DocValues.getNumeric(ctx.reader(), field);
+      return toDoubleValues(values, decoder::applyAsDouble);
+    }
+
+    @Override
+    public boolean needsScores() {
+      return false;
+    }
+  }
+
+  private static class DoubleValuesSortField extends SortField {
+
+    final DoubleValuesSource producer;
+
+    public DoubleValuesSortField(DoubleValuesSource producer, boolean reverse) {
+      super(producer.toString(), new DoubleValuesComparatorSource(producer), reverse);
+      this.producer = producer;
+    }
+
+    @Override
+    public boolean needsScores() {
+      return producer.needsScores();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder buffer = new StringBuilder("<");
+      buffer.append(getField()).append(">");
+      if (reverse)
+        buffer.append("!");
+      return buffer.toString();
+    }
+
+  }
+
+  private static class DoubleValuesHolder {
+    DoubleValues values;
+  }
+
+  private static class DoubleValuesComparatorSource extends FieldComparatorSource {
+    private final DoubleValuesSource producer;
+
+    public DoubleValuesComparatorSource(DoubleValuesSource producer) {
+      this.producer = producer;
+    }
+
+    @Override
+    public FieldComparator<Double> newComparator(String fieldname, int numHits,
+                                               int sortPos, boolean reversed) throws IOException {
+      return new FieldComparator.DoubleComparator(numHits, fieldname, 0.0){
+
+        LeafReaderContext ctx;
+        DoubleValuesHolder holder = new DoubleValuesHolder();
+
+        @Override
+        protected NumericDocValues getNumericDocValues(LeafReaderContext context, String field) throws IOException {
+          ctx = context;
+          return asNumericDocValues(holder, Double::doubleToLongBits);
+        }
+
+        @Override
+        public void setScorer(Scorer scorer) throws IOException {
+          holder.values = producer.getValues(ctx, fromScorer(scorer));
+        }
+      };
+    }
+  }
+
+  private static DoubleValues toDoubleValues(NumericDocValues in, LongToDoubleFunction map) {
+    return new DoubleValues() {
+      @Override
+      public double doubleValue() throws IOException {
+        return map.applyAsDouble(in.longValue());
+      }
+
+      @Override
+      public boolean advanceExact(int target) throws IOException {
+        return in.advanceExact(target);
+      }
+
+    };
+  }
+
+  private static NumericDocValues asNumericDocValues(DoubleValuesHolder in, DoubleToLongFunction converter) {
+    return new NumericDocValues() {
+      @Override
+      public long longValue() throws IOException {
+        return converter.applyAsLong(in.values.doubleValue());
+      }
+
+      @Override
+      public boolean advanceExact(int target) throws IOException {
+        return in.values.advanceExact(target);
+      }
+
+      @Override
+      public int docID() {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public int nextDoc() throws IOException {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public int advance(int target) throws IOException {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public long cost() {
+        throw new UnsupportedOperationException();
+      }
+    };
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3f24fd81/lucene/core/src/java/org/apache/lucene/search/LongValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/LongValues.java b/lucene/core/src/java/org/apache/lucene/search/LongValues.java
new file mode 100644
index 0000000..54d3189
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/LongValues.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search;
+
+import java.io.IOException;
+
+/**
+ * Per-segment, per-document long values, which can be calculated at search-time
+ */
+public abstract class LongValues {
+
+  /**
+   * Get the long value for the current document
+   */
+  public abstract long longValue() throws IOException;
+
+  /**
+   * Advance this instance to the given document id
+   * @return true if there is a value for this document
+   */
+  public abstract boolean advanceExact(int doc) throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3f24fd81/lucene/core/src/java/org/apache/lucene/search/LongValuesSource.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/LongValuesSource.java b/lucene/core/src/java/org/apache/lucene/search/LongValuesSource.java
new file mode 100644
index 0000000..9d00355
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/LongValuesSource.java
@@ -0,0 +1,217 @@
+/*
+ * 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.Objects;
+
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+
+/**
+ * Base class for producing {@link LongValues}
+ *
+ * To obtain a {@link LongValues} object for a leaf reader, clients should
+ * call {@link #getValues(LeafReaderContext, DoubleValues)}.
+ *
+ * LongValuesSource objects for long and int-valued NumericDocValues fields can
+ * be obtained by calling {@link #fromLongField(String)} and {@link #fromIntField(String)}.
+ *
+ * To obtain a LongValuesSource from a float or double-valued NumericDocValues field,
+ * use {@link DoubleValuesSource#fromFloatField(String)} or {@link DoubleValuesSource#fromDoubleField(String)}
+ * and then call {@link DoubleValuesSource#toLongValuesSource()}.
+ */
+public abstract class LongValuesSource {
+
+  /**
+   * Returns a {@link LongValues} instance for the passed-in LeafReaderContext and scores
+   *
+   * If scores are not needed to calculate the values (ie {@link #needsScores() returns false}, callers
+   * may safely pass {@code null} for the {@code scores} parameter.
+   */
+  public abstract LongValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException;
+
+  /**
+   * Return true if document scores are needed to calculate values
+   */
+  public abstract boolean needsScores();
+
+  /**
+   * Create a sort field based on the value of this producer
+   * @param reverse true if the sort should be decreasing
+   */
+  public SortField getSortField(boolean reverse) {
+    return new LongValuesSortField(this, reverse);
+  }
+
+  /**
+   * Creates a LongValuesSource that wraps a long-valued field
+   */
+  public static LongValuesSource fromLongField(String field) {
+    return new FieldValuesSource(field);
+  }
+
+  /**
+   * Creates a LongValuesSource that wraps an int-valued field
+   */
+  public static LongValuesSource fromIntField(String field) {
+    return fromLongField(field);
+  }
+
+  private static class FieldValuesSource extends LongValuesSource {
+
+    final String field;
+
+    private FieldValuesSource(String field) {
+      this.field = field;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+      FieldValuesSource that = (FieldValuesSource) o;
+      return Objects.equals(field, that.field);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(field);
+    }
+
+    @Override
+    public LongValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+      final NumericDocValues values = DocValues.getNumeric(ctx.reader(), field);
+      return toLongValues(values);
+    }
+
+    @Override
+    public boolean needsScores() {
+      return false;
+    }
+  }
+
+  private static class LongValuesSortField extends SortField {
+
+    final LongValuesSource producer;
+
+    public LongValuesSortField(LongValuesSource producer, boolean reverse) {
+      super(producer.toString(), new LongValuesComparatorSource(producer), reverse);
+      this.producer = producer;
+    }
+
+    @Override
+    public boolean needsScores() {
+      return producer.needsScores();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder buffer = new StringBuilder("<");
+      buffer.append(getField()).append(">");
+      if (reverse)
+        buffer.append("!");
+      return buffer.toString();
+    }
+
+  }
+
+  private static class LongValuesHolder {
+    LongValues values;
+  }
+
+  private static class LongValuesComparatorSource extends FieldComparatorSource {
+    private final LongValuesSource producer;
+
+    public LongValuesComparatorSource(LongValuesSource producer) {
+      this.producer = producer;
+    }
+
+    @Override
+    public FieldComparator<Long> newComparator(String fieldname, int numHits,
+                                                 int sortPos, boolean reversed) throws IOException {
+      return new FieldComparator.LongComparator(numHits, fieldname, 0L){
+
+        LeafReaderContext ctx;
+        LongValuesHolder holder = new LongValuesHolder();
+
+        @Override
+        protected NumericDocValues getNumericDocValues(LeafReaderContext context, String field) throws IOException {
+          ctx = context;
+          return asNumericDocValues(holder);
+        }
+
+        @Override
+        public void setScorer(Scorer scorer) throws IOException {
+          holder.values = producer.getValues(ctx, DoubleValuesSource.fromScorer(scorer));
+        }
+      };
+    }
+  }
+
+  private static LongValues toLongValues(NumericDocValues in) {
+    return new LongValues() {
+      @Override
+      public long longValue() throws IOException {
+        return in.longValue();
+      }
+
+      @Override
+      public boolean advanceExact(int target) throws IOException {
+        return in.advanceExact(target);
+      }
+
+    };
+  }
+
+  private static NumericDocValues asNumericDocValues(LongValuesHolder in) {
+    return new NumericDocValues() {
+      @Override
+      public long longValue() throws IOException {
+        return in.values.longValue();
+      }
+
+      @Override
+      public boolean advanceExact(int target) throws IOException {
+        return in.values.advanceExact(target);
+      }
+
+      @Override
+      public int docID() {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public int nextDoc() throws IOException {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public int advance(int target) throws IOException {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public long cost() {
+        throw new UnsupportedOperationException();
+      }
+    };
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3f24fd81/lucene/core/src/test/org/apache/lucene/search/TestDoubleValuesSource.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestDoubleValuesSource.java b/lucene/core/src/test/org/apache/lucene/search/TestDoubleValuesSource.java
new file mode 100644
index 0000000..3860963
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/TestDoubleValuesSource.java
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search;
+
+import java.util.Arrays;
+import java.util.Collections;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.DoubleDocValuesField;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FloatDocValuesField;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.English;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+public class TestDoubleValuesSource extends LuceneTestCase {
+
+  private Directory dir;
+  private IndexReader reader;
+  private IndexSearcher searcher;
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    int numDocs = TestUtil.nextInt(random(), 2049, 4000);
+    for (int i = 0; i < numDocs; i++) {
+      Document document = new Document();
+      document.add(newTextField("english", English.intToEnglish(i), Field.Store.NO));
+      document.add(newTextField("oddeven", (i % 2 == 0) ? "even" : "odd", Field.Store.NO));
+      document.add(new NumericDocValuesField("int", random().nextInt()));
+      document.add(new NumericDocValuesField("long", random().nextLong()));
+      document.add(new FloatDocValuesField("float", random().nextFloat()));
+      document.add(new DoubleDocValuesField("double", random().nextDouble()));
+      iw.addDocument(document);
+    }
+    reader = iw.getReader();
+    iw.close();
+    searcher = newSearcher(reader);
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    reader.close();
+    dir.close();
+    super.tearDown();
+  }
+
+  public void testSimpleFieldEquivalences() throws Exception {
+    checkSorts(new MatchAllDocsQuery(), new Sort(new SortField("int", SortField.Type.INT, random().nextBoolean())));
+    checkSorts(new MatchAllDocsQuery(), new Sort(new SortField("long", SortField.Type.LONG, random().nextBoolean())));
+    checkSorts(new MatchAllDocsQuery(), new Sort(new SortField("float", SortField.Type.FLOAT, random().nextBoolean())));
+    checkSorts(new MatchAllDocsQuery(), new Sort(new SortField("double", SortField.Type.DOUBLE, random().nextBoolean())));
+  }
+
+  public void testHashCodeAndEquals() {
+    DoubleValuesSource vs1 = DoubleValuesSource.fromDoubleField("double");
+    DoubleValuesSource vs2 = DoubleValuesSource.fromDoubleField("double");
+    assertEquals(vs1, vs2);
+    assertEquals(vs1.hashCode(), vs2.hashCode());
+    DoubleValuesSource v3 = DoubleValuesSource.fromLongField("long");
+    assertFalse(vs1.equals(v3));
+  }
+
+  public void testSimpleFieldSortables() throws Exception {
+    int n = atLeast(4);
+    for (int i = 0; i < n; i++) {
+      Sort sort = randomSort();
+      checkSorts(new MatchAllDocsQuery(), sort);
+      checkSorts(new TermQuery(new Term("english", "one")), sort);
+    }
+  }
+
+  Sort randomSort() throws Exception {
+    boolean reversed = random().nextBoolean();
+    SortField fields[] = new SortField[] {
+        new SortField("int", SortField.Type.INT, reversed),
+        new SortField("long", SortField.Type.LONG, reversed),
+        new SortField("float", SortField.Type.FLOAT, reversed),
+        new SortField("double", SortField.Type.DOUBLE, reversed),
+        new SortField("score", SortField.Type.SCORE)
+    };
+    Collections.shuffle(Arrays.asList(fields), random());
+    int numSorts = TestUtil.nextInt(random(), 1, fields.length);
+    return new Sort(Arrays.copyOfRange(fields, 0, numSorts));
+  }
+
+  // Take a Sort, and replace any field sorts with Sortables
+  Sort convertSortToSortable(Sort sort) {
+    SortField original[] = sort.getSort();
+    SortField mutated[] = new SortField[original.length];
+    for (int i = 0; i < mutated.length; i++) {
+      if (random().nextInt(3) > 0) {
+        SortField s = original[i];
+        boolean reverse = s.getType() == SortField.Type.SCORE || s.getReverse();
+        switch (s.getType()) {
+          case INT:
+            mutated[i] = DoubleValuesSource.fromIntField(s.getField()).getSortField(reverse);
+            break;
+          case LONG:
+            mutated[i] = DoubleValuesSource.fromLongField(s.getField()).getSortField(reverse);
+            break;
+          case FLOAT:
+            mutated[i] = DoubleValuesSource.fromFloatField(s.getField()).getSortField(reverse);
+            break;
+          case DOUBLE:
+            mutated[i] = DoubleValuesSource.fromDoubleField(s.getField()).getSortField(reverse);
+            break;
+          case SCORE:
+            mutated[i] = DoubleValuesSource.SCORES.getSortField(reverse);
+            break;
+          default:
+            mutated[i] = original[i];
+        }
+      } else {
+        mutated[i] = original[i];
+      }
+    }
+
+    return new Sort(mutated);
+  }
+
+  void checkSorts(Query query, Sort sort) throws Exception {
+    int size = TestUtil.nextInt(random(), 1, searcher.getIndexReader().maxDoc() / 5);
+    TopDocs expected = searcher.search(query, size, sort, random().nextBoolean(), random().nextBoolean());
+    Sort mutatedSort = convertSortToSortable(sort);
+    TopDocs actual = searcher.search(query, size, mutatedSort, random().nextBoolean(), random().nextBoolean());
+
+    CheckHits.checkEqual(query, expected.scoreDocs, actual.scoreDocs);
+
+    if (size < actual.totalHits) {
+      expected = searcher.searchAfter(expected.scoreDocs[size-1], query, size, sort);
+      actual = searcher.searchAfter(actual.scoreDocs[size-1], query, size, mutatedSort);
+      CheckHits.checkEqual(query, expected.scoreDocs, actual.scoreDocs);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3f24fd81/lucene/core/src/test/org/apache/lucene/search/TestLongValuesSource.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestLongValuesSource.java b/lucene/core/src/test/org/apache/lucene/search/TestLongValuesSource.java
new file mode 100644
index 0000000..9148ad5
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/TestLongValuesSource.java
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search;
+
+import java.util.Arrays;
+import java.util.Collections;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.English;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+public class TestLongValuesSource extends LuceneTestCase {
+
+  private Directory dir;
+  private IndexReader reader;
+  private IndexSearcher searcher;
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    int numDocs = TestUtil.nextInt(random(), 2049, 4000);
+    for (int i = 0; i < numDocs; i++) {
+      Document document = new Document();
+      document.add(newTextField("english", English.intToEnglish(i), Field.Store.NO));
+      document.add(newTextField("oddeven", (i % 2 == 0) ? "even" : "odd", Field.Store.NO));
+      document.add(new NumericDocValuesField("int", random().nextInt()));
+      document.add(new NumericDocValuesField("long", random().nextLong()));
+      iw.addDocument(document);
+    }
+    reader = iw.getReader();
+    iw.close();
+    searcher = newSearcher(reader);
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    reader.close();
+    dir.close();
+    super.tearDown();
+  }
+
+  public void testSimpleFieldEquivalences() throws Exception {
+    checkSorts(new MatchAllDocsQuery(), new Sort(new SortField("int", SortField.Type.INT, random().nextBoolean())));
+    checkSorts(new MatchAllDocsQuery(), new Sort(new SortField("long", SortField.Type.LONG, random().nextBoolean())));
+  }
+
+  public void testHashCodeAndEquals() {
+    LongValuesSource vs1 = LongValuesSource.fromLongField("long");
+    LongValuesSource vs2 = LongValuesSource.fromLongField("long");
+    assertEquals(vs1, vs2);
+    assertEquals(vs1.hashCode(), vs2.hashCode());
+    LongValuesSource v3 = LongValuesSource.fromLongField("int");
+    assertFalse(vs1.equals(v3));
+  }
+
+  public void testSimpleFieldSortables() throws Exception {
+    int n = atLeast(4);
+    for (int i = 0; i < n; i++) {
+      Sort sort = randomSort();
+      checkSorts(new MatchAllDocsQuery(), sort);
+      checkSorts(new TermQuery(new Term("english", "one")), sort);
+    }
+  }
+
+  Sort randomSort() throws Exception {
+    boolean reversed = random().nextBoolean();
+    SortField fields[] = new SortField[] {
+        new SortField("int", SortField.Type.INT, reversed),
+        new SortField("long", SortField.Type.LONG, reversed)
+    };
+    Collections.shuffle(Arrays.asList(fields), random());
+    int numSorts = TestUtil.nextInt(random(), 1, fields.length);
+    return new Sort(Arrays.copyOfRange(fields, 0, numSorts));
+  }
+
+  // Take a Sort, and replace any field sorts with Sortables
+  Sort convertSortToSortable(Sort sort) {
+    SortField original[] = sort.getSort();
+    SortField mutated[] = new SortField[original.length];
+    for (int i = 0; i < mutated.length; i++) {
+      if (random().nextInt(3) > 0) {
+        SortField s = original[i];
+        boolean reverse = s.getType() == SortField.Type.SCORE || s.getReverse();
+        switch (s.getType()) {
+          case INT:
+            mutated[i] = LongValuesSource.fromIntField(s.getField()).getSortField(reverse);
+            break;
+          case LONG:
+            mutated[i] = LongValuesSource.fromLongField(s.getField()).getSortField(reverse);
+            break;
+          default:
+            mutated[i] = original[i];
+        }
+      } else {
+        mutated[i] = original[i];
+      }
+    }
+
+    return new Sort(mutated);
+  }
+
+  void checkSorts(Query query, Sort sort) throws Exception {
+    int size = TestUtil.nextInt(random(), 1, searcher.getIndexReader().maxDoc() / 5);
+    Sort mutatedSort = convertSortToSortable(sort);
+    TopDocs actual = searcher.search(query, size, mutatedSort, random().nextBoolean(), random().nextBoolean());
+    TopDocs expected = searcher.search(query, size, sort, random().nextBoolean(), random().nextBoolean());
+
+    CheckHits.checkEqual(query, expected.scoreDocs, actual.scoreDocs);
+
+    if (size < actual.totalHits) {
+      expected = searcher.searchAfter(expected.scoreDocs[size-1], query, size, sort);
+      actual = searcher.searchAfter(actual.scoreDocs[size-1], query, size, mutatedSort);
+      CheckHits.checkEqual(query, expected.scoreDocs, actual.scoreDocs);
+    }
+  }
+}


[44/50] lucene-solr:jira/solr-8593: LUCENE-7603: add package-info.java for new package

Posted by kr...@apache.org.
LUCENE-7603: add package-info.java for new package


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

Branch: refs/heads/jira/solr-8593
Commit: 018df31da8b6b5beeb767c90d7ef2a784eca354a
Parents: 3c96396
Author: Mike McCandless <mi...@apache.org>
Authored: Tue Jan 3 05:31:56 2017 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Tue Jan 3 05:31:56 2017 -0500

----------------------------------------------------------------------
 .../apache/lucene/util/graph/package-info.java  | 21 ++++++++++++++++++++
 1 file changed, 21 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/018df31d/lucene/core/src/java/org/apache/lucene/util/graph/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/graph/package-info.java b/lucene/core/src/java/org/apache/lucene/util/graph/package-info.java
new file mode 100644
index 0000000..bbc9b6a
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/util/graph/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+
+/**
+ * Utility classes for working with token streams as graphs.
+ */
+package org.apache.lucene.util.graph;


[42/50] lucene-solr:jira/solr-8593: LUCENE-7603: add CHANGES entry

Posted by kr...@apache.org.
LUCENE-7603: add CHANGES entry


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

Branch: refs/heads/jira/solr-8593
Commit: 1bcf9a251d597cdc029295325b287ce5ce661bec
Parents: a81ebce
Author: Mike McCandless <mi...@apache.org>
Authored: Tue Jan 3 05:15:37 2017 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Tue Jan 3 05:15:37 2017 -0500

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


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1bcf9a25/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 4c49560..474080c 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -94,6 +94,10 @@ New features
   expose per-segment LongValues or DoubleValues iterators, similar to the
   existing DocValues iterator API. (Alan Woodward, Adrien Grand)
 
+* LUCENE-7603: Graph token streams are now handled accurately by query
+  parsers, by enumerating all paths and creating the corresponding
+  query/ies as sub-clauses (Matt Weber via Mike McCandless)
+
 Bug Fixes
 
 * LUCENE-7547: JapaneseTokenizerFactory was failing to close the


[16/50] lucene-solr:jira/solr-8593: LUCENE-7607: FieldLeafComparator.setScorer() should throw IOException

Posted by kr...@apache.org.
LUCENE-7607: FieldLeafComparator.setScorer() should throw IOException


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

Branch: refs/heads/jira/solr-8593
Commit: 87b6c2c8fcdc3a5f4adc3516f249af89b479d77a
Parents: 12aff1c
Author: Alan Woodward <ro...@apache.org>
Authored: Wed Dec 28 19:48:16 2016 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Thu Dec 29 12:22:40 2016 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                                    |  3 +++
 .../org/apache/lucene/search/LeafFieldComparator.java |  2 +-
 .../apache/lucene/search/SimpleFieldComparator.java   |  2 +-
 .../apache/solr/search/CollapsingQParserPlugin.java   | 14 +++++++-------
 4 files changed, 12 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/87b6c2c8/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 7a118f1..35314db 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -65,6 +65,9 @@ API Changes
 * LUCENE-7533: Classic query parser no longer allows autoGeneratePhraseQueries
   to be set to true when splitOnWhitespace is false (and vice-versa).
 
+* LUCENE-7607: LeafFieldComparator.setScorer and SimpleFieldComparator.setScorer
+  are declared as throwing IOException (Alan Woodward)
+
 New features
 
 * LUCENE-5867: Added BooleanSimilarity. (Robert Muir, Adrien Grand)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/87b6c2c8/lucene/core/src/java/org/apache/lucene/search/LeafFieldComparator.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/LeafFieldComparator.java b/lucene/core/src/java/org/apache/lucene/search/LeafFieldComparator.java
index 6019376..677639c 100644
--- a/lucene/core/src/java/org/apache/lucene/search/LeafFieldComparator.java
+++ b/lucene/core/src/java/org/apache/lucene/search/LeafFieldComparator.java
@@ -114,6 +114,6 @@ public interface LeafFieldComparator {
    * 
    * @param scorer Scorer instance that you should use to
    * obtain the current hit's score, if necessary. */
-  void setScorer(Scorer scorer);
+  void setScorer(Scorer scorer) throws IOException;
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/87b6c2c8/lucene/core/src/java/org/apache/lucene/search/SimpleFieldComparator.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/SimpleFieldComparator.java b/lucene/core/src/java/org/apache/lucene/search/SimpleFieldComparator.java
index 3f1e6c2..a258e24 100644
--- a/lucene/core/src/java/org/apache/lucene/search/SimpleFieldComparator.java
+++ b/lucene/core/src/java/org/apache/lucene/search/SimpleFieldComparator.java
@@ -38,5 +38,5 @@ public abstract class SimpleFieldComparator<T> extends FieldComparator<T> implem
   }
 
   @Override
-  public void setScorer(Scorer scorer) {}
+  public void setScorer(Scorer scorer) throws IOException {}
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/87b6c2c8/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
index f6bc18e..44aade5 100644
--- a/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
+++ b/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
@@ -965,7 +965,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
 
     @Override public boolean needsScores() { return needsScores || super.needsScores(); }
 
-    public void setScorer(Scorer scorer) {
+    public void setScorer(Scorer scorer) throws IOException {
       this.collapseStrategy.setScorer(scorer);
     }
 
@@ -1147,7 +1147,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
 
     @Override public boolean needsScores() { return needsScores || super.needsScores(); }
 
-    public void setScorer(Scorer scorer) {
+    public void setScorer(Scorer scorer) throws IOException {
       this.collapseStrategy.setScorer(scorer);
     }
 
@@ -1523,7 +1523,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
       return collapsedSet;
     }
 
-    public void setScorer(Scorer scorer) {
+    public void setScorer(Scorer scorer) throws IOException {
       this.scorer = scorer;
     }
 
@@ -1952,7 +1952,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
     }
 
     @Override
-    public void setScorer(Scorer s) {
+    public void setScorer(Scorer s) throws IOException {
       super.setScorer(s);
       this.compareState.setScorer(s);
     }
@@ -2100,7 +2100,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
       return collapsedSet;
     }
 
-    public void setScorer(Scorer scorer) {
+    public void setScorer(Scorer scorer) throws IOException {
       this.scorer = scorer;
     }
 
@@ -2522,7 +2522,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
     }
 
     @Override
-    public void setScorer(Scorer s) {
+    public void setScorer(Scorer s) throws IOException {
       super.setScorer(s);
       this.compareState.setScorer(s);
     }
@@ -2673,7 +2673,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
         leafFieldComparators[clause] = fieldComparators[clause].getLeafComparator(context);
       }
     }
-    public void setScorer(Scorer s) {
+    public void setScorer(Scorer s) throws IOException {
       for (int clause = 0; clause < numClauses; clause++) {
         leafFieldComparators[clause].setScorer(s);
       }


[10/50] lucene-solr:jira/solr-8593: SOLR-9901: Implement move in HdfsDirectoryFactory.

Posted by kr...@apache.org.
SOLR-9901: Implement move in HdfsDirectoryFactory.


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

Branch: refs/heads/jira/solr-8593
Commit: 197590a928cfefa51b1a8307046e5a11e5400e34
Parents: a5e5c4a
Author: markrmiller <ma...@apache.org>
Authored: Wed Dec 28 16:16:14 2016 -0500
Committer: markrmiller <ma...@apache.org>
Committed: Thu Dec 29 05:47:41 2016 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  2 ++
 .../apache/solr/core/HdfsDirectoryFactory.java  | 20 ++++++++++++++++++++
 2 files changed, 22 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/197590a9/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 06566e0..1383859 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -294,6 +294,8 @@ Bug Fixes
 * SOLR-9859: replication.properties cannot be updated after being written and neither replication.properties or 
   index.properties are durable in the face of a crash. (Pushkar Raste, Chris de Kok, Cao Manh Dat, Mark Miller)
 
+* SOLR-9901: Implement move in HdfsDirectoryFactory. (Mark Miller)
+
 Other Changes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/197590a9/solr/core/src/java/org/apache/solr/core/HdfsDirectoryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/HdfsDirectoryFactory.java b/solr/core/src/java/org/apache/solr/core/HdfsDirectoryFactory.java
index d481e03..e1e3d6e 100644
--- a/solr/core/src/java/org/apache/solr/core/HdfsDirectoryFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/HdfsDirectoryFactory.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.LockFactory;
 import org.apache.lucene.store.NRTCachingDirectory;
 import org.apache.lucene.store.NoLockFactory;
@@ -577,4 +578,23 @@ public class HdfsDirectoryFactory extends CachingDirectoryFactory implements Sol
     FileContext fileContext = FileContext.getFileContext(getConf());
     fileContext.rename(new Path(hdfsDirPath + "/" + fileName), new Path(hdfsDirPath + "/" + toName), Options.Rename.OVERWRITE);
   }
+  
+  @Override
+  public void move(Directory fromDir, Directory toDir, String fileName, IOContext ioContext) throws IOException {
+    
+    Directory baseFromDir = getBaseDir(fromDir);
+    Directory baseToDir = getBaseDir(toDir);
+    
+    if (baseFromDir instanceof HdfsDirectory && baseToDir instanceof HdfsDirectory) {
+      Path dir1 = ((HdfsDirectory) baseFromDir).getHdfsDirPath();
+      Path dir2 = ((HdfsDirectory) baseToDir).getHdfsDirPath();
+      Path file1 = new Path(dir1, fileName);
+      Path file2 = new Path(dir2, fileName);
+      FileContext fileContext = FileContext.getFileContext(getConf());
+      fileContext.rename(file1, file2);
+      return;
+    }
+
+    super.move(fromDir, toDir, fileName, ioContext);
+  }
 }


[49/50] lucene-solr:jira/solr-8593: LUCENE-7612: Remove suggester dependency on misc

Posted by kr...@apache.org.
LUCENE-7612: Remove suggester dependency on misc


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

Branch: refs/heads/jira/solr-8593
Commit: f3306786a13bda596a60cada45406e6f6a88724f
Parents: f6fb694
Author: Alan Woodward <ro...@apache.org>
Authored: Fri Dec 30 14:45:29 2016 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Tue Jan 3 13:56:00 2017 +0000

----------------------------------------------------------------------
 dev-tools/idea/lucene/suggest/suggest.iml | 1 -
 lucene/CHANGES.txt                        | 3 +++
 lucene/suggest/build.xml                  | 6 ++----
 3 files changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f3306786/dev-tools/idea/lucene/suggest/suggest.iml
----------------------------------------------------------------------
diff --git a/dev-tools/idea/lucene/suggest/suggest.iml b/dev-tools/idea/lucene/suggest/suggest.iml
index ef2b8ed..576ea8b 100644
--- a/dev-tools/idea/lucene/suggest/suggest.iml
+++ b/dev-tools/idea/lucene/suggest/suggest.iml
@@ -15,7 +15,6 @@
     <orderEntry type="module" scope="TEST" module-name="lucene-test-framework" />
     <orderEntry type="module" module-name="queries" />
     <orderEntry type="module" module-name="analysis-common" />
-    <orderEntry type="module" module-name="misc" />
     <orderEntry type="module" module-name="lucene-core" />
   </component>
 </module>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f3306786/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 5b7a10c..67d8ae5 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -141,6 +141,9 @@ Bug Fixes
 * LUCENE-7606: Normalization with CustomAnalyzer would only apply the last
   token filter. (Adrien Grand)
 
+* LUCENE-7612: Removed an unused dependency from the suggester to the misc
+  module. (Alan Woodward)
+
 Improvements
 
 * LUCENE-6824: TermAutomatonQuery now rewrites to TermQuery,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f3306786/lucene/suggest/build.xml
----------------------------------------------------------------------
diff --git a/lucene/suggest/build.xml b/lucene/suggest/build.xml
index 26b316b..5babe06 100644
--- a/lucene/suggest/build.xml
+++ b/lucene/suggest/build.xml
@@ -30,22 +30,20 @@
 
   <path id="classpath">
     <pathelement path="${analyzers-common.jar}"/>
-    <pathelement path="${misc.jar}"/>
     <pathelement path="${queries.jar}"/>
     <path refid="base.classpath"/>
   </path>
 
-  <target name="javadocs" depends="javadocs-queries,javadocs-misc,compile-core,check-javadocs-uptodate"
+  <target name="javadocs" depends="javadocs-queries,compile-core,check-javadocs-uptodate"
           unless="javadocs-uptodate-${name}">
     <invoke-module-javadoc>
       <links>
         <link href="../analyzers-common"/>
         <link href="../queries"/>
-        <link href="../misc"/>
       </links>
     </invoke-module-javadoc>
   </target>
 
-  <target name="compile-core" depends="jar-expressions, jar-misc, jar-analyzers-common, common.compile-core" />
+  <target name="compile-core" depends="jar-expressions, jar-analyzers-common, common.compile-core" />
 
 </project>


[40/50] lucene-solr:jira/solr-8593: SOLR-9906: unused import

Posted by kr...@apache.org.
SOLR-9906: unused import


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

Branch: refs/heads/jira/solr-8593
Commit: d5652385675d12b80a58e44a8c8b392c9f70a334
Parents: 33304f2
Author: Noble Paul <no...@apache.org>
Authored: Tue Jan 3 20:38:28 2017 +1030
Committer: Noble Paul <no...@apache.org>
Committed: Tue Jan 3 20:38:28 2017 +1030

----------------------------------------------------------------------
 .../src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java   | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d5652385/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java
index 7f991a4..0669cbe 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java
@@ -19,7 +19,6 @@ package org.apache.solr.cloud;
 import java.io.File;
 import java.lang.invoke.MethodHandles;
 import java.util.Map;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.io.FileUtils;


[43/50] lucene-solr:jira/solr-8593: SOLR-9896: Instrument and collect metrics from query, update, core admin and core load thread pools

Posted by kr...@apache.org.
SOLR-9896: Instrument and collect metrics from query, update, core admin and core load thread pools


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

Branch: refs/heads/jira/solr-8593
Commit: 3c963967242aed73a906b7bc17c26a4b8b07083c
Parents: 1bcf9a2
Author: Shalin Shekhar Mangar <sh...@apache.org>
Authored: Tue Jan 3 15:52:01 2017 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Tue Jan 3 15:52:01 2017 +0530

----------------------------------------------------------------------
 solr/CHANGES.txt                                     |  2 ++
 .../src/java/org/apache/solr/core/CoreContainer.java | 15 ++++++++++++---
 .../apache/solr/handler/admin/CoreAdminHandler.java  | 13 ++++++++++++-
 .../handler/component/HttpShardHandlerFactory.java   | 10 +++++++---
 .../org/apache/solr/update/UpdateShardHandler.java   |  9 ++++++++-
 .../java/org/apache/solr/util/stats/MetricUtils.java |  9 +++++++++
 6 files changed, 50 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3c963967/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 6326e54..8609f91 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -210,6 +210,8 @@ New Features
 
 * SOLR-9684: Add priority Streaming Expression (Joel Bernstein, David Smiley)
 
+* SOLR-9896: Instrument and collect metrics from query, update, core admin and core load thread pools. (shalin)
+
 Optimizations
 ----------------------
 * SOLR-9704: Facet Module / JSON Facet API: Optimize blockChildren facets that have

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3c963967/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 f3747dc..de7c34d 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -78,6 +78,7 @@ import org.apache.solr.security.SecurityPluginHolder;
 import org.apache.solr.update.SolrCoreState;
 import org.apache.solr.update.UpdateShardHandler;
 import org.apache.solr.util.DefaultSolrThreadFactory;
+import org.apache.solr.util.stats.MetricUtils;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -462,6 +463,11 @@ public class CoreContainer {
 
     metricManager = new SolrMetricManager();
 
+    coreContainerWorkExecutor = MetricUtils.instrumentedExecutorService(
+        coreContainerWorkExecutor,
+        metricManager.registry(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node)),
+        SolrMetricManager.mkName("coreContainerWorkExecutor", "threadPool"));
+
     shardHandlerFactory = ShardHandlerFactory.newInstance(cfg.getShardHandlerFactoryPluginInfo(), loader);
     if (shardHandlerFactory instanceof SolrMetricProducer) {
       SolrMetricProducer metricProducer = (SolrMetricProducer) shardHandlerFactory;
@@ -520,9 +526,12 @@ public class CoreContainer {
         unloadedCores, true, "unloaded", "cores");
 
     // setup executor to load cores in parallel
-    ExecutorService coreLoadExecutor = ExecutorUtil.newMDCAwareFixedThreadPool(
-        cfg.getCoreLoadThreadCount(isZooKeeperAware()),
-        new DefaultSolrThreadFactory("coreLoadExecutor") );
+    ExecutorService coreLoadExecutor = MetricUtils.instrumentedExecutorService(
+        ExecutorUtil.newMDCAwareFixedThreadPool(
+            cfg.getCoreLoadThreadCount(isZooKeeperAware()),
+            new DefaultSolrThreadFactory("coreLoadExecutor")),
+        metricManager.registry(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node)),
+        SolrMetricManager.mkName("coreLoadExecutor", "threadPool"));
     final List<Future<SolrCore>> futures = new ArrayList<>();
     try {
       List<CoreDescriptor> cds = coresLocator.discover(this);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3c963967/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
index 7b0ecfb..458b7a5 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
@@ -18,6 +18,7 @@ package org.apache.solr.handler.admin;
 
 import java.io.File;
 import java.lang.invoke.MethodHandles;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -42,11 +43,13 @@ import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.handler.RequestHandlerBase;
+import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.security.AuthorizationContext;
 import org.apache.solr.security.PermissionNameProvider;
 import org.apache.solr.util.DefaultSolrThreadFactory;
+import org.apache.solr.util.stats.MetricUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.MDC;
@@ -65,7 +68,7 @@ public class CoreAdminHandler extends RequestHandlerBase implements PermissionNa
   protected final CoreContainer coreContainer;
   protected final Map<String, Map<String, TaskObject>> requestStatusMap;
 
-  protected final ExecutorService parallelExecutor = ExecutorUtil.newMDCAwareFixedThreadPool(50,
+  protected ExecutorService parallelExecutor = ExecutorUtil.newMDCAwareFixedThreadPool(50,
       new DefaultSolrThreadFactory("parallelCoreAdminExecutor"));
 
   protected static int MAX_TRACKED_REQUESTS = 100;
@@ -111,6 +114,14 @@ public class CoreAdminHandler extends RequestHandlerBase implements PermissionNa
                     "it is a special Handler configured directly by the RequestDispatcher");
   }
 
+  @Override
+  public Collection<String> initializeMetrics(SolrMetricManager manager, String registryName, String scope) {
+    Collection<String> metrics = super.initializeMetrics(manager, registryName, scope);
+    parallelExecutor = MetricUtils.instrumentedExecutorService(parallelExecutor, manager.registry(registryName),
+        SolrMetricManager.mkName("parallelCoreAdminExecutor", getCategory().name(),scope, "threadPool"));
+    return metrics;
+  }
+
   /**
    * The instance of CoreContainer this handler handles. This should be the CoreContainer instance that created this
    * handler.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3c963967/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
index 3c01720..d190ce0 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
@@ -42,6 +42,7 @@ import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.util.DefaultSolrThreadFactory;
 import org.apache.solr.util.stats.InstrumentedHttpRequestExecutor;
 import org.apache.solr.util.stats.InstrumentedPoolingHttpClientConnectionManager;
+import org.apache.solr.util.stats.MetricUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -58,8 +59,8 @@ import java.util.concurrent.ArrayBlockingQueue;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.CompletionService;
 import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
 import java.util.concurrent.SynchronousQueue;
-import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 
 
@@ -73,7 +74,7 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
   //
   // Consider CallerRuns policy and a lower max threads to throttle
   // requests at some point (or should we simply return failure?)
-  private ThreadPoolExecutor commExecutor = new ExecutorUtil.MDCAwareThreadPoolExecutor(
+  private ExecutorService commExecutor = new ExecutorUtil.MDCAwareThreadPoolExecutor(
       0,
       Integer.MAX_VALUE,
       5, TimeUnit.SECONDS, // terminate idle threads after 5 sec
@@ -191,7 +192,7 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
     return clientParams;
   }
 
-  protected ThreadPoolExecutor getThreadPoolExecutor(){
+  protected ExecutorService getThreadPoolExecutor(){
     return this.commExecutor;
   }
 
@@ -378,6 +379,9 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
     List<String> metricNames = new ArrayList<>(4);
     metricNames.addAll(clientConnectionManager.initializeMetrics(manager, registry, scope));
     metricNames.addAll(httpRequestExecutor.initializeMetrics(manager, registry, scope));
+    commExecutor = MetricUtils.instrumentedExecutorService(commExecutor,
+        manager.registry(registry),
+        SolrMetricManager.mkName("httpShardExecutor", scope, "threadPool"));
     return metricNames;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3c963967/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java b/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
index c3ed8cd..9d230bc 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
@@ -23,6 +23,7 @@ import java.util.Collection;
 import java.util.List;
 import java.util.concurrent.ExecutorService;
 
+import com.codahale.metrics.InstrumentedExecutorService;
 import org.apache.http.client.HttpClient;
 import org.apache.http.impl.client.CloseableHttpClient;
 import org.apache.http.impl.conn.PoolingHttpClientConnectionManager;
@@ -102,12 +103,18 @@ public class UpdateShardHandler implements SolrMetricProducer {
     List<String> metricNames = new ArrayList<>(4);
     metricNames.addAll(clientConnectionManager.initializeMetrics(manager, registry, scope));
     metricNames.addAll(httpRequestExecutor.initializeMetrics(manager, registry, scope));
+    updateExecutor = new InstrumentedExecutorService(updateExecutor,
+        manager.registry(registry),
+        SolrMetricManager.mkName("updateExecutor", scope, "threadPool"));
+    recoveryExecutor = new InstrumentedExecutorService(recoveryExecutor,
+        manager.registry(registry),
+        SolrMetricManager.mkName("recoveryExecutor", scope, "threadPool"));
     return metricNames;
   }
 
   @Override
   public String getDescription() {
-    return "Metrics tracked by UpdateShardHandler for ";
+    return "Metrics tracked by UpdateShardHandler related to distributed updates and recovery";
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3c963967/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 62f5776..af5a0b5 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
@@ -19,11 +19,13 @@ package org.apache.solr.util.stats;
 import java.util.List;
 import java.util.Map;
 import java.util.SortedSet;
+import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 
 import com.codahale.metrics.Counter;
 import com.codahale.metrics.Gauge;
 import com.codahale.metrics.Histogram;
+import com.codahale.metrics.InstrumentedExecutorService;
 import com.codahale.metrics.Meter;
 import com.codahale.metrics.Metric;
 import com.codahale.metrics.MetricFilter;
@@ -141,4 +143,11 @@ public class MetricUtils {
     response.add("requests", counter.getCount());
     return response;
   }
+
+  /**
+   * Returns an instrumented wrapper over the given executor service.
+   */
+  public static ExecutorService instrumentedExecutorService(ExecutorService delegate, MetricRegistry metricRegistry, String scope)  {
+    return new InstrumentedExecutorService(delegate, metricRegistry, scope);
+  }
 }


[47/50] lucene-solr:jira/solr-8593: LUCENE-7588: DrillSideways can now run its queries concurrently

Posted by kr...@apache.org.
LUCENE-7588: DrillSideways can now run its queries concurrently


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

Branch: refs/heads/jira/solr-8593
Commit: b4a002f7d88a2383852e2bfd95b39bf7f6e33f2f
Parents: 48ca9fc
Author: Mike McCandless <mi...@apache.org>
Authored: Tue Jan 3 06:26:49 2017 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Tue Jan 3 06:26:49 2017 -0500

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   5 +
 .../lucene/search/MultiCollectorManager.java    | 105 ++++++
 .../org/apache/lucene/facet/DrillSideways.java  | 338 +++++++++++++++----
 .../lucene/facet/FacetsCollectorManager.java    |  55 +++
 .../apache/lucene/facet/TestDrillSideways.java  | 315 +++++++++--------
 .../lucene/facet/TestParallelDrillSideways.java |  90 +++++
 6 files changed, 707 insertions(+), 201 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4a002f7/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 474080c..5b7a10c 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -98,6 +98,11 @@ New features
   parsers, by enumerating all paths and creating the corresponding
   query/ies as sub-clauses (Matt Weber via Mike McCandless)
 
+* LUCENE-7588: DrillSideways can now run queries concurrently, and
+  supports an IndexSearcher using an executor service to run each query
+  concurrently across all segments in the index (Emmanuel Keller via
+  Mike McCandless)
+
 Bug Fixes
 
 * LUCENE-7547: JapaneseTokenizerFactory was failing to close the

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4a002f7/lucene/core/src/java/org/apache/lucene/search/MultiCollectorManager.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/MultiCollectorManager.java b/lucene/core/src/java/org/apache/lucene/search/MultiCollectorManager.java
new file mode 100644
index 0000000..9549cde
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/MultiCollectorManager.java
@@ -0,0 +1,105 @@
+/*
+ * 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 org.apache.lucene.index.LeafReaderContext;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * A {@link CollectorManager} implements which wrap a set of {@link CollectorManager}
+ * as {@link MultiCollector} acts for {@link Collector}.
+ */
+public class MultiCollectorManager implements CollectorManager<MultiCollectorManager.Collectors, Object[]> {
+
+  final private CollectorManager<Collector, ?>[] collectorManagers;
+
+  public MultiCollectorManager(final CollectorManager... collectorManagers) {
+    this.collectorManagers = collectorManagers;
+  }
+
+  @Override
+  public Collectors newCollector() throws IOException {
+    return new Collectors();
+  }
+
+  @Override
+  public Object[] reduce(Collection<Collectors> reducableCollectors) throws IOException {
+    final int size = reducableCollectors.size();
+    final Object[] results = new Object[collectorManagers.length];
+    for (int i = 0; i < collectorManagers.length; i++) {
+      final List<Collector> reducableCollector = new ArrayList<>(size);
+      for (Collectors collectors : reducableCollectors)
+        reducableCollector.add(collectors.collectors[i]);
+      results[i] = collectorManagers[i].reduce(reducableCollector);
+    }
+    return results;
+  }
+
+  public class Collectors implements Collector {
+
+    private final Collector[] collectors;
+
+    private Collectors() throws IOException {
+      collectors = new Collector[collectorManagers.length];
+      for (int i = 0; i < collectors.length; i++)
+        collectors[i] = collectorManagers[i].newCollector();
+    }
+
+    @Override
+    final public LeafCollector getLeafCollector(final LeafReaderContext context) throws IOException {
+      return new LeafCollectors(context);
+    }
+
+    @Override
+    final public boolean needsScores() {
+      for (Collector collector : collectors)
+        if (collector.needsScores())
+          return true;
+      return false;
+    }
+
+    public class LeafCollectors implements LeafCollector {
+
+      private final LeafCollector[] leafCollectors;
+
+      private LeafCollectors(final LeafReaderContext context) throws IOException {
+        leafCollectors = new LeafCollector[collectors.length];
+        for (int i = 0; i < collectors.length; i++)
+          leafCollectors[i] = collectors[i].getLeafCollector(context);
+      }
+
+      @Override
+      final public void setScorer(final Scorer scorer) throws IOException {
+        for (LeafCollector leafCollector : leafCollectors)
+          if (leafCollector != null)
+            leafCollector.setScorer(scorer);
+      }
+
+      @Override
+      final public void collect(final int doc) throws IOException {
+        for (LeafCollector leafCollector : leafCollectors)
+          if (leafCollector != null)
+            leafCollector.collect(doc);
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4a002f7/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java b/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java
index 57f0a32..61530bc 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java
@@ -16,39 +16,47 @@
  */
 package org.apache.lucene.facet;
 
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-
 import org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetCounts;
 import org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetField;
 import org.apache.lucene.facet.sortedset.SortedSetDocValuesReaderState;
 import org.apache.lucene.facet.taxonomy.FastTaxonomyFacetCounts;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
-import org.apache.lucene.search.FilterCollector;
 import org.apache.lucene.search.Collector;
+import org.apache.lucene.search.CollectorManager;
 import org.apache.lucene.search.FieldDoc;
+import org.apache.lucene.search.FilterCollector;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.MultiCollector;
+import org.apache.lucene.search.MultiCollectorManager;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.search.TopFieldCollector;
 import org.apache.lucene.search.TopScoreDocCollector;
+import org.apache.lucene.util.ThreadInterruptedException;
 
-/**     
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
+/**
  * Computes drill down and sideways counts for the provided
  * {@link DrillDownQuery}.  Drill sideways counts include
  * alternative values/aggregates for the drill-down
  * dimensions so that a dimension does not disappear after
  * the user drills down into it.
- *
  * <p> Use one of the static search
  * methods to do the search, and then get the hits and facet
  * results from the returned {@link DrillSidewaysResult}.
- *
  * <p><b>NOTE</b>: this allocates one {@link
  * FacetsCollector} for each drill-down, plus one.  If your
  * index has high number of facet labels then this will
@@ -58,62 +66,96 @@ import org.apache.lucene.search.TopScoreDocCollector;
  */
 public class DrillSideways {
 
-  /** {@link IndexSearcher} passed to constructor. */
+  /**
+   * {@link IndexSearcher} passed to constructor.
+   */
   protected final IndexSearcher searcher;
 
-  /** {@link TaxonomyReader} passed to constructor. */
+  /**
+   * {@link TaxonomyReader} passed to constructor.
+   */
   protected final TaxonomyReader taxoReader;
 
-  /** {@link SortedSetDocValuesReaderState} passed to
-   *  constructor; can be null. */
+  /**
+   * {@link SortedSetDocValuesReaderState} passed to
+   * constructor; can be null.
+   */
   protected final SortedSetDocValuesReaderState state;
 
-  /** {@link FacetsConfig} passed to constructor. */
+  /**
+   * {@link FacetsConfig} passed to constructor.
+   */
   protected final FacetsConfig config;
 
-  /** Create a new {@code DrillSideways} instance. */
+  // These are only used for multi-threaded search
+  private final ExecutorService executor;
+
+  /**
+   * Create a new {@code DrillSideways} instance.
+   */
   public DrillSideways(IndexSearcher searcher, FacetsConfig config, TaxonomyReader taxoReader) {
     this(searcher, config, taxoReader, null);
   }
-    
-  /** Create a new {@code DrillSideways} instance, assuming the categories were
-   *  indexed with {@link SortedSetDocValuesFacetField}. */
+
+  /**
+   * Create a new {@code DrillSideways} instance, assuming the categories were
+   * indexed with {@link SortedSetDocValuesFacetField}.
+   */
   public DrillSideways(IndexSearcher searcher, FacetsConfig config, SortedSetDocValuesReaderState state) {
     this(searcher, config, null, state);
   }
 
-  /** Create a new {@code DrillSideways} instance, where some
-   *  dimensions were indexed with {@link
-   *  SortedSetDocValuesFacetField} and others were indexed
-   *  with {@link FacetField}. */
-  public DrillSideways(IndexSearcher searcher, FacetsConfig config, TaxonomyReader taxoReader, SortedSetDocValuesReaderState state) {
+  /**
+   * Create a new {@code DrillSideways} instance, where some
+   * dimensions were indexed with {@link
+   * SortedSetDocValuesFacetField} and others were indexed
+   * with {@link FacetField}.
+   */
+  public DrillSideways(IndexSearcher searcher, FacetsConfig config, TaxonomyReader taxoReader,
+          SortedSetDocValuesReaderState state) {
+    this(searcher, config, taxoReader, state, null);
+  }
+
+  /**
+   * Create a new {@code DrillSideways} instance, where some
+   * dimensions were indexed with {@link
+   * SortedSetDocValuesFacetField} and others were indexed
+   * with {@link FacetField}.
+   * <p>
+   * Use this constructor to use the concurrent implementation and/or the CollectorManager
+   */
+  public DrillSideways(IndexSearcher searcher, FacetsConfig config, TaxonomyReader taxoReader,
+          SortedSetDocValuesReaderState state, ExecutorService executor) {
     this.searcher = searcher;
     this.config = config;
     this.taxoReader = taxoReader;
     this.state = state;
+    this.executor = executor;
   }
 
-  /** Subclass can override to customize per-dim Facets
-   *  impl. */
-  protected Facets buildFacetsResult(FacetsCollector drillDowns, FacetsCollector[] drillSideways, String[] drillSidewaysDims) throws IOException {
+  /**
+   * Subclass can override to customize per-dim Facets
+   * impl.
+   */
+  protected Facets buildFacetsResult(FacetsCollector drillDowns, FacetsCollector[] drillSideways,
+          String[] drillSidewaysDims) throws IOException {
 
     Facets drillDownFacets;
-    Map<String,Facets> drillSidewaysFacets = new HashMap<>();
+    Map<String, Facets> drillSidewaysFacets = new HashMap<>();
 
     if (taxoReader != null) {
       drillDownFacets = new FastTaxonomyFacetCounts(taxoReader, config, drillDowns);
       if (drillSideways != null) {
-        for(int i=0;i<drillSideways.length;i++) {
+        for (int i = 0; i < drillSideways.length; i++) {
           drillSidewaysFacets.put(drillSidewaysDims[i],
-                                  new FastTaxonomyFacetCounts(taxoReader, config, drillSideways[i]));
+                  new FastTaxonomyFacetCounts(taxoReader, config, drillSideways[i]));
         }
       }
     } else {
       drillDownFacets = new SortedSetDocValuesFacetCounts(state, drillDowns);
       if (drillSideways != null) {
-        for(int i=0;i<drillSideways.length;i++) {
-          drillSidewaysFacets.put(drillSidewaysDims[i],
-                                  new SortedSetDocValuesFacetCounts(state, drillSideways[i]));
+        for (int i = 0; i < drillSideways.length; i++) {
+          drillSidewaysFacets.put(drillSidewaysDims[i], new SortedSetDocValuesFacetCounts(state, drillSideways[i]));
         }
       }
     }
@@ -131,10 +173,10 @@ public class DrillSideways {
    */
   public DrillSidewaysResult search(DrillDownQuery query, Collector hitCollector) throws IOException {
 
-    Map<String,Integer> drillDownDims = query.getDims();
+    Map<String, Integer> drillDownDims = query.getDims();
 
     FacetsCollector drillDownCollector = new FacetsCollector();
-    
+
     if (drillDownDims.isEmpty()) {
       // There are no drill-down dims, so there is no
       // drill-sideways to compute:
@@ -154,8 +196,10 @@ public class DrillSideways {
     for (int i = 0; i < drillSidewaysCollectors.length; i++) {
       drillSidewaysCollectors[i] = new FacetsCollector();
     }
-    
-    DrillSidewaysQuery dsq = new DrillSidewaysQuery(baseQuery, drillDownCollector, drillSidewaysCollectors, drillDownQueries, scoreSubDocsAtOnce());
+
+    DrillSidewaysQuery dsq =
+            new DrillSidewaysQuery(baseQuery, drillDownCollector, drillSidewaysCollectors, drillDownQueries,
+                    scoreSubDocsAtOnce());
     if (hitCollector.needsScores() == false) {
       // this is a horrible hack in order to make sure IndexSearcher will not
       // attempt to cache the DrillSidewaysQuery
@@ -168,16 +212,16 @@ public class DrillSideways {
     }
     searcher.search(dsq, hitCollector);
 
-    return new DrillSidewaysResult(buildFacetsResult(drillDownCollector, drillSidewaysCollectors, drillDownDims.keySet().toArray(new String[drillDownDims.size()])), null);
+    return new DrillSidewaysResult(buildFacetsResult(drillDownCollector, drillSidewaysCollectors,
+            drillDownDims.keySet().toArray(new String[drillDownDims.size()])), null);
   }
 
   /**
    * Search, sorting by {@link Sort}, and computing
    * drill down and sideways counts.
    */
-  public DrillSidewaysResult search(DrillDownQuery query,
-                                    Query filter, FieldDoc after, int topN, Sort sort, boolean doDocScores,
-                                    boolean doMaxScore) throws IOException {
+  public DrillSidewaysResult search(DrillDownQuery query, Query filter, FieldDoc after, int topN, Sort sort,
+          boolean doDocScores, boolean doMaxScore) throws IOException {
     if (filter != null) {
       query = new DrillDownQuery(config, filter, query);
     }
@@ -186,15 +230,38 @@ public class DrillSideways {
       if (limit == 0) {
         limit = 1; // the collector does not alow numHits = 0
       }
-      topN = Math.min(topN, limit);
-      final TopFieldCollector hitCollector = TopFieldCollector.create(sort,
-                                                                      topN,
-                                                                      after,
-                                                                      true,
-                                                                      doDocScores,
-                                                                      doMaxScore);
-      DrillSidewaysResult r = search(query, hitCollector);
-      return new DrillSidewaysResult(r.facets, hitCollector.topDocs());
+      final int fTopN = Math.min(topN, limit);
+
+      if (executor != null) { // We have an executor, let use the multi-threaded version
+
+        final CollectorManager<TopFieldCollector, TopDocs> collectorManager =
+                new CollectorManager<TopFieldCollector, TopDocs>() {
+
+                  @Override
+                  public TopFieldCollector newCollector() throws IOException {
+                    return TopFieldCollector.create(sort, fTopN, after, true, doDocScores, doMaxScore);
+                  }
+
+                  @Override
+                  public TopDocs reduce(Collection<TopFieldCollector> collectors) throws IOException {
+                    final TopDocs[] topDocs = new TopDocs[collectors.size()];
+                    int pos = 0;
+                    for (TopFieldCollector collector : collectors)
+                      topDocs[pos++] = collector.topDocs();
+                    return TopDocs.merge(topN, topDocs);
+                  }
+
+                };
+        ConcurrentDrillSidewaysResult<TopDocs> r = search(query, collectorManager);
+        return new DrillSidewaysResult(r.facets, r.collectorResult);
+
+      } else {
+
+        final TopFieldCollector hitCollector =
+                TopFieldCollector.create(sort, fTopN, after, true, doDocScores, doMaxScore);
+        DrillSidewaysResult r = search(query, hitCollector);
+        return new DrillSidewaysResult(r.facets, hitCollector.topDocs());
+      }
     } else {
       return search(after, query, topN);
     }
@@ -212,41 +279,184 @@ public class DrillSideways {
    * Search, sorting by score, and computing
    * drill down and sideways counts.
    */
-  public DrillSidewaysResult search(ScoreDoc after,
-                                    DrillDownQuery query, int topN) throws IOException {
+  public DrillSidewaysResult search(ScoreDoc after, DrillDownQuery query, int topN) throws IOException {
     int limit = searcher.getIndexReader().maxDoc();
     if (limit == 0) {
       limit = 1; // the collector does not alow numHits = 0
     }
-    topN = Math.min(topN, limit);
-    TopScoreDocCollector hitCollector = TopScoreDocCollector.create(topN, after);
-    DrillSidewaysResult r = search(query, hitCollector);
-    return new DrillSidewaysResult(r.facets, hitCollector.topDocs());
+    final int fTopN = Math.min(topN, limit);
+
+    if (executor != null) {  // We have an executor, let use the multi-threaded version
+
+      final CollectorManager<TopScoreDocCollector, TopDocs> collectorManager =
+              new CollectorManager<TopScoreDocCollector, TopDocs>() {
+
+                @Override
+                public TopScoreDocCollector newCollector() throws IOException {
+                  return TopScoreDocCollector.create(fTopN, after);
+                }
+
+                @Override
+                public TopDocs reduce(Collection<TopScoreDocCollector> collectors) throws IOException {
+                  final TopDocs[] topDocs = new TopDocs[collectors.size()];
+                  int pos = 0;
+                  for (TopScoreDocCollector collector : collectors)
+                    topDocs[pos++] = collector.topDocs();
+                  return TopDocs.merge(topN, topDocs);
+                }
+
+              };
+      ConcurrentDrillSidewaysResult<TopDocs> r = search(query, collectorManager);
+      return new DrillSidewaysResult(r.facets, r.collectorResult);
+
+    } else {
+
+      TopScoreDocCollector hitCollector = TopScoreDocCollector.create(topN, after);
+      DrillSidewaysResult r = search(query, hitCollector);
+      return new DrillSidewaysResult(r.facets, hitCollector.topDocs());
+    }
   }
 
-  /** Override this and return true if your collector
-   *  (e.g., {@code ToParentBlockJoinCollector}) expects all
-   *  sub-scorers to be positioned on the document being
-   *  collected.  This will cause some performance loss;
-   *  default is false. */
+  /**
+   * Override this and return true if your collector
+   * (e.g., {@code ToParentBlockJoinCollector}) expects all
+   * sub-scorers to be positioned on the document being
+   * collected.  This will cause some performance loss;
+   * default is false.
+   */
   protected boolean scoreSubDocsAtOnce() {
     return false;
   }
 
-  /** Result of a drill sideways search, including the
-   *  {@link Facets} and {@link TopDocs}. */
+  /**
+   * Result of a drill sideways search, including the
+   * {@link Facets} and {@link TopDocs}.
+   */
   public static class DrillSidewaysResult {
-    /** Combined drill down and sideways results. */
+    /**
+     * Combined drill down and sideways results.
+     */
     public final Facets facets;
 
-    /** Hits. */
+    /**
+     * Hits.
+     */
     public final TopDocs hits;
 
-    /** Sole constructor. */
+    /**
+     * Sole constructor.
+     */
     public DrillSidewaysResult(Facets facets, TopDocs hits) {
       this.facets = facets;
       this.hits = hits;
     }
   }
+
+  private static class CallableCollector implements Callable<CallableResult> {
+
+    private final int pos;
+    private final IndexSearcher searcher;
+    private final Query query;
+    private final CollectorManager<?, ?> collectorManager;
+
+    private CallableCollector(int pos, IndexSearcher searcher, Query query, CollectorManager<?, ?> collectorManager) {
+      this.pos = pos;
+      this.searcher = searcher;
+      this.query = query;
+      this.collectorManager = collectorManager;
+    }
+
+    @Override
+    public CallableResult call() throws Exception {
+      return new CallableResult(pos, searcher.search(query, collectorManager));
+    }
+  }
+
+  private static class CallableResult {
+
+    private final int pos;
+    private final Object result;
+
+    private CallableResult(int pos, Object result) {
+      this.pos = pos;
+      this.result = result;
+    }
+  }
+
+  private DrillDownQuery getDrillDownQuery(final DrillDownQuery query, Query[] queries,
+          final String excludedDimension) {
+    final DrillDownQuery ddl = new DrillDownQuery(config, query.getBaseQuery());
+    query.getDims().forEach((dim, pos) -> {
+      if (!dim.equals(excludedDimension))
+        ddl.add(dim, queries[pos]);
+    });
+    return ddl.getDims().size() == queries.length ? null : ddl;
+  }
+
+  /** Runs a search, using a {@link CollectorManager} to gather and merge search results */
+  public <R> ConcurrentDrillSidewaysResult<R> search(final DrillDownQuery query,
+          final CollectorManager<?, R> hitCollectorManager) throws IOException {
+
+    final Map<String, Integer> drillDownDims = query.getDims();
+    final List<CallableCollector> callableCollectors = new ArrayList<>(drillDownDims.size() + 1);
+
+    // Add the main DrillDownQuery
+    callableCollectors.add(new CallableCollector(-1, searcher, query,
+            new MultiCollectorManager(new FacetsCollectorManager(), hitCollectorManager)));
+    int i = 0;
+    final Query[] filters = query.getDrillDownQueries();
+    for (String dim : drillDownDims.keySet())
+      callableCollectors.add(new CallableCollector(i++, searcher, getDrillDownQuery(query, filters, dim),
+              new FacetsCollectorManager()));
+
+    final FacetsCollector mainFacetsCollector;
+    final FacetsCollector[] facetsCollectors = new FacetsCollector[drillDownDims.size()];
+    final R collectorResult;
+
+    try {
+      // Run the query pool
+      final List<Future<CallableResult>> futures = executor.invokeAll(callableCollectors);
+
+      // Extract the results
+      final Object[] mainResults = (Object[]) futures.get(0).get().result;
+      mainFacetsCollector = (FacetsCollector) mainResults[0];
+      collectorResult = (R) mainResults[1];
+      for (i = 1; i < futures.size(); i++) {
+        final CallableResult result = futures.get(i).get();
+        facetsCollectors[result.pos] = (FacetsCollector) result.result;
+      }
+      // Fill the null results with the mainFacetsCollector
+      for (i = 0; i < facetsCollectors.length; i++)
+        if (facetsCollectors[i] == null)
+          facetsCollectors[i] = mainFacetsCollector;
+
+    } catch (InterruptedException e) {
+      throw new ThreadInterruptedException(e);
+    } catch (ExecutionException e) {
+      throw new RuntimeException(e);
+    }
+
+    // build the facets and return the result
+    return new ConcurrentDrillSidewaysResult<>(buildFacetsResult(mainFacetsCollector, facetsCollectors,
+            drillDownDims.keySet().toArray(new String[drillDownDims.size()])), null, collectorResult);
+  }
+
+  /**
+   * Result of a concurrent drill sideways search, including the
+   * {@link Facets} and {@link TopDocs}.
+   */
+  public static class ConcurrentDrillSidewaysResult<R> extends DrillSidewaysResult {
+
+    /** The merged search results */
+    public final R collectorResult;
+
+    /**
+     * Sole constructor.
+     */
+    ConcurrentDrillSidewaysResult(Facets facets, TopDocs hits, R collectorResult) {
+      super(facets, hits);
+      this.collectorResult = collectorResult;
+    }
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4a002f7/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollectorManager.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollectorManager.java b/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollectorManager.java
new file mode 100644
index 0000000..652436d
--- /dev/null
+++ b/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollectorManager.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.facet;
+
+import org.apache.lucene.search.CollectorManager;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * A {@link CollectorManager} implementation which produce FacetsCollector and product a merged FacetsCollector.
+ * This is used for concurrent FacetsCollection.
+ */
+class FacetsCollectorManager implements CollectorManager<FacetsCollector, FacetsCollector> {
+
+  public final static FacetsCollector EMPTY = new FacetsCollector();
+
+  @Override
+  public FacetsCollector newCollector() throws IOException {
+    return new FacetsCollector();
+  }
+
+  @Override
+  public FacetsCollector reduce(Collection<FacetsCollector> collectors) throws IOException {
+    if (collectors == null || collectors.size() == 0)
+      return EMPTY;
+    if (collectors.size() == 1)
+      return collectors.iterator().next();
+    return new ReducedFacetsCollector(collectors);
+  }
+
+  private static class ReducedFacetsCollector extends FacetsCollector {
+
+    public ReducedFacetsCollector(final Collection<FacetsCollector> facetsCollectors) {
+      final List<MatchingDocs> matchingDocs = this.getMatchingDocs();
+      facetsCollectors.forEach(facetsCollector -> matchingDocs.addAll(facetsCollector.getMatchingDocs()));
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4a002f7/lucene/facet/src/test/org/apache/lucene/facet/TestDrillSideways.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/TestDrillSideways.java b/lucene/facet/src/test/org/apache/lucene/facet/TestDrillSideways.java
index 3960956..ff3e2fc 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/TestDrillSideways.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/TestDrillSideways.java
@@ -16,16 +16,6 @@
  */
 package org.apache.lucene.facet;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-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 org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
@@ -68,8 +58,61 @@ import org.apache.lucene.util.InPlaceMergeSorter;
 import org.apache.lucene.util.InfoStream;
 import org.apache.lucene.util.TestUtil;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
 public class TestDrillSideways extends FacetTestCase {
 
+  protected DrillSideways getNewDrillSideways(IndexSearcher searcher, FacetsConfig config,
+          SortedSetDocValuesReaderState state) {
+    return new DrillSideways(searcher, config, state);
+  }
+
+  protected DrillSideways getNewDrillSideways(IndexSearcher searcher, FacetsConfig config, TaxonomyReader taxoReader) {
+    return new DrillSideways(searcher, config, taxoReader);
+  }
+
+  protected DrillSideways getNewDrillSidewaysScoreSubdocsAtOnce(IndexSearcher searcher, FacetsConfig config,
+          TaxonomyReader taxoReader) {
+    return new DrillSideways(searcher, config, taxoReader) {
+      @Override
+      protected boolean scoreSubDocsAtOnce() {
+        return true;
+      }
+    };
+  }
+
+  protected DrillSideways getNewDrillSidewaysBuildFacetsResult(IndexSearcher searcher, FacetsConfig config,
+          TaxonomyReader taxoReader) {
+    return new DrillSideways(searcher, config, taxoReader) {
+      @Override
+      protected Facets buildFacetsResult(FacetsCollector drillDowns, FacetsCollector[] drillSideways,
+              String[] drillSidewaysDims) throws IOException {
+        Map<String, Facets> drillSidewaysFacets = new HashMap<>();
+        Facets drillDownFacets = getTaxonomyFacetCounts(taxoReader, config, drillDowns);
+        if (drillSideways != null) {
+          for (int i = 0; i < drillSideways.length; i++) {
+            drillSidewaysFacets.put(drillSidewaysDims[i], getTaxonomyFacetCounts(taxoReader, config, drillSideways[i]));
+          }
+        }
+
+        if (drillSidewaysFacets.isEmpty()) {
+          return drillDownFacets;
+        } else {
+          return new MultiFacets(drillSidewaysFacets, drillDownFacets);
+        }
+
+      }
+    };
+  }
+
   public void testBasic() throws Exception {
     Directory dir = newDirectory();
     Directory taxoDir = newDirectory();
@@ -116,7 +159,7 @@ public class TestDrillSideways extends FacetTestCase {
     // NRT open
     TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoWriter);
 
-    DrillSideways ds = new DrillSideways(searcher, config, taxoReader);
+    DrillSideways ds = getNewDrillSideways(searcher, config, taxoReader);
 
     //  case: drill-down on a single field; in this
     // case the drill-sideways + drill-down counts ==
@@ -127,12 +170,14 @@ public class TestDrillSideways extends FacetTestCase {
     assertEquals(2, r.hits.totalHits);
     // Publish Date is only drill-down, and Lisa published
     // one in 2012 and one in 2010:
-    assertEquals("dim=Publish Date path=[] value=2 childCount=2\n  2010 (1)\n  2012 (1)\n", r.facets.getTopChildren(10, "Publish Date").toString());
+    assertEquals("dim=Publish Date path=[] value=2 childCount=2\n  2010 (1)\n  2012 (1)\n",
+            r.facets.getTopChildren(10, "Publish Date").toString());
 
     // Author is drill-sideways + drill-down: Lisa
     // (drill-down) published twice, and Frank/Susan/Bob
     // published once:
-    assertEquals("dim=Author path=[] value=5 childCount=4\n  Lisa (2)\n  Bob (1)\n  Susan (1)\n  Frank (1)\n", r.facets.getTopChildren(10, "Author").toString());
+    assertEquals("dim=Author path=[] value=5 childCount=4\n  Lisa (2)\n  Bob (1)\n  Susan (1)\n  Frank (1)\n",
+            r.facets.getTopChildren(10, "Author").toString());
 
     // Same simple case, but no baseQuery (pure browse):
     // drill-down on a single field; in this case the
@@ -145,12 +190,14 @@ public class TestDrillSideways extends FacetTestCase {
     assertEquals(2, r.hits.totalHits);
     // Publish Date is only drill-down, and Lisa published
     // one in 2012 and one in 2010:
-    assertEquals("dim=Publish Date path=[] value=2 childCount=2\n  2010 (1)\n  2012 (1)\n", r.facets.getTopChildren(10, "Publish Date").toString());
+    assertEquals("dim=Publish Date path=[] value=2 childCount=2\n  2010 (1)\n  2012 (1)\n",
+            r.facets.getTopChildren(10, "Publish Date").toString());
 
     // Author is drill-sideways + drill-down: Lisa
     // (drill-down) published twice, and Frank/Susan/Bob
     // published once:
-    assertEquals("dim=Author path=[] value=5 childCount=4\n  Lisa (2)\n  Bob (1)\n  Susan (1)\n  Frank (1)\n", r.facets.getTopChildren(10, "Author").toString());
+    assertEquals("dim=Author path=[] value=5 childCount=4\n  Lisa (2)\n  Bob (1)\n  Susan (1)\n  Frank (1)\n",
+            r.facets.getTopChildren(10, "Author").toString());
 
     // Another simple case: drill-down on single fields
     // but OR of two values
@@ -161,17 +208,21 @@ public class TestDrillSideways extends FacetTestCase {
     assertEquals(3, r.hits.totalHits);
     // Publish Date is only drill-down: Lisa and Bob
     // (drill-down) published twice in 2010 and once in 2012:
-    assertEquals("dim=Publish Date path=[] value=3 childCount=2\n  2010 (2)\n  2012 (1)\n", r.facets.getTopChildren(10, "Publish Date").toString());
+    assertEquals("dim=Publish Date path=[] value=3 childCount=2\n  2010 (2)\n  2012 (1)\n",
+            r.facets.getTopChildren(10, "Publish Date").toString());
     // Author is drill-sideways + drill-down: Lisa
     // (drill-down) published twice, and Frank/Susan/Bob
     // published once:
-    assertEquals("dim=Author path=[] value=5 childCount=4\n  Lisa (2)\n  Bob (1)\n  Susan (1)\n  Frank (1)\n", r.facets.getTopChildren(10, "Author").toString());
+    assertEquals("dim=Author path=[] value=5 childCount=4\n  Lisa (2)\n  Bob (1)\n  Susan (1)\n  Frank (1)\n",
+            r.facets.getTopChildren(10, "Author").toString());
 
     assertTrue(r.facets instanceof MultiFacets);
     List<FacetResult> allResults = r.facets.getAllDims(10);
     assertEquals(2, allResults.size());
-    assertEquals("dim=Author path=[] value=5 childCount=4\n  Lisa (2)\n  Bob (1)\n  Susan (1)\n  Frank (1)\n", allResults.get(0).toString());
-    assertEquals("dim=Publish Date path=[] value=3 childCount=2\n  2010 (2)\n  2012 (1)\n", allResults.get(1).toString());
+    assertEquals("dim=Author path=[] value=5 childCount=4\n  Lisa (2)\n  Bob (1)\n  Susan (1)\n  Frank (1)\n",
+            allResults.get(0).toString());
+    assertEquals("dim=Publish Date path=[] value=3 childCount=2\n  2010 (2)\n  2012 (1)\n",
+            allResults.get(1).toString());
 
     // More interesting case: drill-down on two fields
     ddq = new DrillDownQuery(config);
@@ -181,10 +232,12 @@ public class TestDrillSideways extends FacetTestCase {
     assertEquals(1, r.hits.totalHits);
     // Publish Date is drill-sideways + drill-down: Lisa
     // (drill-down) published once in 2010 and once in 2012:
-    assertEquals("dim=Publish Date path=[] value=2 childCount=2\n  2010 (1)\n  2012 (1)\n", r.facets.getTopChildren(10, "Publish Date").toString());
+    assertEquals("dim=Publish Date path=[] value=2 childCount=2\n  2010 (1)\n  2012 (1)\n",
+            r.facets.getTopChildren(10, "Publish Date").toString());
     // Author is drill-sideways + drill-down:
     // only Lisa & Bob published (once each) in 2010:
-    assertEquals("dim=Author path=[] value=2 childCount=2\n  Bob (1)\n  Lisa (1)\n", r.facets.getTopChildren(10, "Author").toString());
+    assertEquals("dim=Author path=[] value=2 childCount=2\n  Bob (1)\n  Lisa (1)\n",
+            r.facets.getTopChildren(10, "Author").toString());
 
     // Even more interesting case: drill down on two fields,
     // but one of them is OR
@@ -198,10 +251,12 @@ public class TestDrillSideways extends FacetTestCase {
     assertEquals(2, r.hits.totalHits);
     // Publish Date is both drill-sideways + drill-down:
     // Lisa or Bob published twice in 2010 and once in 2012:
-    assertEquals("dim=Publish Date path=[] value=3 childCount=2\n  2010 (2)\n  2012 (1)\n", r.facets.getTopChildren(10, "Publish Date").toString());
+    assertEquals("dim=Publish Date path=[] value=3 childCount=2\n  2010 (2)\n  2012 (1)\n",
+            r.facets.getTopChildren(10, "Publish Date").toString());
     // Author is drill-sideways + drill-down:
     // only Lisa & Bob published (once each) in 2010:
-    assertEquals("dim=Author path=[] value=2 childCount=2\n  Bob (1)\n  Lisa (1)\n", r.facets.getTopChildren(10, "Author").toString());
+    assertEquals("dim=Author path=[] value=2 childCount=2\n  Bob (1)\n  Lisa (1)\n",
+            r.facets.getTopChildren(10, "Author").toString());
 
     // Test drilling down on invalid field:
     ddq = new DrillDownQuery(config);
@@ -219,11 +274,13 @@ public class TestDrillSideways extends FacetTestCase {
     assertEquals(2, r.hits.totalHits);
     // Publish Date is only drill-down, and Lisa published
     // one in 2012 and one in 2010:
-    assertEquals("dim=Publish Date path=[] value=2 childCount=2\n  2010 (1)\n  2012 (1)\n", r.facets.getTopChildren(10, "Publish Date").toString());
+    assertEquals("dim=Publish Date path=[] value=2 childCount=2\n  2010 (1)\n  2012 (1)\n",
+            r.facets.getTopChildren(10, "Publish Date").toString());
     // Author is drill-sideways + drill-down: Lisa
     // (drill-down) published twice, and Frank/Susan/Bob
     // published once:
-    assertEquals("dim=Author path=[] value=5 childCount=4\n  Lisa (2)\n  Bob (1)\n  Susan (1)\n  Frank (1)\n", r.facets.getTopChildren(10, "Author").toString());
+    assertEquals("dim=Author path=[] value=5 childCount=4\n  Lisa (2)\n  Bob (1)\n  Susan (1)\n  Frank (1)\n",
+            r.facets.getTopChildren(10, "Author").toString());
 
     // LUCENE-4915: test drilling down on a dimension but
     // NOT facet counting it:
@@ -234,7 +291,8 @@ public class TestDrillSideways extends FacetTestCase {
     assertEquals(2, r.hits.totalHits);
     // Publish Date is only drill-down, and Lisa published
     // one in 2012 and one in 2010:
-    assertEquals("dim=Publish Date path=[] value=2 childCount=2\n  2010 (1)\n  2012 (1)\n", r.facets.getTopChildren(10, "Publish Date").toString());
+    assertEquals("dim=Publish Date path=[] value=2 childCount=2\n  2010 (1)\n  2012 (1)\n",
+            r.facets.getTopChildren(10, "Publish Date").toString());
 
     // Test main query gets null scorer:
     ddq = new DrillDownQuery(config, new TermQuery(new Term("foobar", "baz")));
@@ -288,16 +346,18 @@ public class TestDrillSideways extends FacetTestCase {
 
     DrillDownQuery ddq = new DrillDownQuery(config);
     ddq.add("Author", "Lisa");
-    DrillSidewaysResult r = new DrillSideways(searcher, config, taxoReader).search(null, ddq, 10);
+    DrillSidewaysResult r = getNewDrillSideways(searcher, config, taxoReader).search(null, ddq, 10);
 
     assertEquals(1, r.hits.totalHits);
     // Publish Date is only drill-down, and Lisa published
     // one in 2012 and one in 2010:
-    assertEquals("dim=Publish Date path=[] value=1 childCount=1\n  2010 (1)\n", r.facets.getTopChildren(10, "Publish Date").toString());
+    assertEquals("dim=Publish Date path=[] value=1 childCount=1\n  2010 (1)\n",
+            r.facets.getTopChildren(10, "Publish Date").toString());
     // Author is drill-sideways + drill-down: Lisa
     // (drill-down) published once, and Bob
     // published once:
-    assertEquals("dim=Author path=[] value=2 childCount=2\n  Bob (1)\n  Lisa (1)\n", r.facets.getTopChildren(10, "Author").toString());
+    assertEquals("dim=Author path=[] value=2 childCount=2\n  Bob (1)\n  Lisa (1)\n",
+            r.facets.getTopChildren(10, "Author").toString());
 
     writer.close();
     IOUtils.close(searcher.getIndexReader(), taxoReader, taxoWriter, dir, taxoDir);
@@ -349,11 +409,13 @@ public class TestDrillSideways extends FacetTestCase {
 
     DrillDownQuery ddq = new DrillDownQuery(config);
     ddq.add("dim", "a");
-    DrillSidewaysResult r = new DrillSideways(searcher, config, taxoReader).search(null, ddq, 10);
+    DrillSidewaysResult r = getNewDrillSideways(searcher, config, taxoReader).search(null, ddq, 10);
 
     assertEquals(3, r.hits.totalHits);
-    assertEquals("dim=dim path=[] value=6 childCount=4\n  a (3)\n  b (1)\n  c (1)\n  d (1)\n", r.facets.getTopChildren(10, "dim").toString());
-    assertEquals("dim=dim path=[a] value=3 childCount=3\n  x (1)\n  y (1)\n  z (1)\n", r.facets.getTopChildren(10, "dim", "a").toString());
+    assertEquals("dim=dim path=[] value=6 childCount=4\n  a (3)\n  b (1)\n  c (1)\n  d (1)\n",
+            r.facets.getTopChildren(10, "dim").toString());
+    assertEquals("dim=dim path=[a] value=3 childCount=3\n  x (1)\n  y (1)\n  z (1)\n",
+            r.facets.getTopChildren(10, "dim", "a").toString());
 
     writer.close();
     IOUtils.close(searcher.getIndexReader(), taxoReader, taxoWriter, dir, taxoDir);
@@ -363,7 +425,8 @@ public class TestDrillSideways extends FacetTestCase {
     String id;
     String contentToken;
 
-    public Doc() {}
+    public Doc() {
+    }
 
     // -1 if the doc is missing this dim, else the index
     // -into the values for this dim:
@@ -427,12 +490,14 @@ public class TestDrillSideways extends FacetTestCase {
     int numDocs = atLeast(3000);
     //int numDocs = 20;
     if (VERBOSE) {
-      System.out.println("numDims=" + numDims + " numDocs=" + numDocs + " aChance=" + aChance + " bChance=" + bChance + " cChance=" + cChance);
+      System.out.println(
+              "numDims=" + numDims + " numDocs=" + numDocs + " aChance=" + aChance + " bChance=" + bChance + " cChance="
+                      + cChance);
     }
     String[][] dimValues = new String[numDims][];
     int valueCount = 2;
 
-    for(int dim=0;dim<numDims;dim++) {
+    for (int dim = 0; dim < numDims; dim++) {
       Set<String> values = new HashSet<>();
       while (values.size() < valueCount) {
         String s = TestUtil.randomRealisticUnicodeString(random());
@@ -446,19 +511,19 @@ public class TestDrillSideways extends FacetTestCase {
     }
 
     List<Doc> docs = new ArrayList<>();
-    for(int i=0;i<numDocs;i++) {
+    for (int i = 0; i < numDocs; i++) {
       Doc doc = new Doc();
-      doc.id = ""+i;
+      doc.id = "" + i;
       doc.contentToken = randomContentToken(false);
       doc.dims = new int[numDims];
       doc.dims2 = new int[numDims];
-      for(int dim=0;dim<numDims;dim++) {
+      for (int dim = 0; dim < numDims; dim++) {
         if (random().nextInt(5) == 3) {
           // This doc is missing this dim:
           doc.dims[dim] = -1;
         } else if (dimValues[dim].length <= 4) {
           int dimUpto = 0;
-          doc.dims[dim] = dimValues[dim].length-1;
+          doc.dims[dim] = dimValues[dim].length - 1;
           while (dimUpto < dimValues[dim].length) {
             if (random().nextBoolean()) {
               doc.dims[dim] = dimUpto;
@@ -488,13 +553,13 @@ public class TestDrillSideways extends FacetTestCase {
     RandomIndexWriter w = new RandomIndexWriter(random(), d, iwc);
     DirectoryTaxonomyWriter tw = new DirectoryTaxonomyWriter(td, IndexWriterConfig.OpenMode.CREATE);
     FacetsConfig config = new FacetsConfig();
-    for(int i=0;i<numDims;i++) {
-      config.setMultiValued("dim"+i, true);
+    for (int i = 0; i < numDims; i++) {
+      config.setMultiValued("dim" + i, true);
     }
 
     boolean doUseDV = random().nextBoolean();
 
-    for(Doc rawDoc : docs) {
+    for (Doc rawDoc : docs) {
       Document doc = new Document();
       doc.add(newStringField("id", rawDoc.id, Field.Store.YES));
       doc.add(new SortedDocValuesField("id", new BytesRef(rawDoc.id)));
@@ -503,7 +568,7 @@ public class TestDrillSideways extends FacetTestCase {
       if (VERBOSE) {
         System.out.println("  doc id=" + rawDoc.id + " token=" + rawDoc.contentToken);
       }
-      for(int dim=0;dim<numDims;dim++) {
+      for (int dim = 0; dim < numDims; dim++) {
         int dimValue = rawDoc.dims[dim];
         if (dimValue != -1) {
           if (doUseDV) {
@@ -579,12 +644,13 @@ public class TestDrillSideways extends FacetTestCase {
 
     int numIters = atLeast(10);
 
-    for(int iter=0;iter<numIters;iter++) {
+    for (int iter = 0; iter < numIters; iter++) {
 
       String contentToken = random().nextInt(30) == 17 ? null : randomContentToken(true);
       int numDrillDown = TestUtil.nextInt(random(), 1, Math.min(4, numDims));
       if (VERBOSE) {
-        System.out.println("\nTEST: iter=" + iter + " baseQuery=" + contentToken + " numDrillDown=" + numDrillDown + " useSortedSetDV=" + doUseDV);
+        System.out.println("\nTEST: iter=" + iter + " baseQuery=" + contentToken + " numDrillDown=" + numDrillDown
+                + " useSortedSetDV=" + doUseDV);
       }
 
       String[][] drillDowns = new String[numDims][];
@@ -596,15 +662,15 @@ public class TestDrillSideways extends FacetTestCase {
         if (drillDowns[dim] == null) {
           if (random().nextBoolean()) {
             // Drill down on one value:
-            drillDowns[dim] = new String[] {dimValues[dim][random().nextInt(dimValues[dim].length)]};
+            drillDowns[dim] = new String[] { dimValues[dim][random().nextInt(dimValues[dim].length)] };
           } else {
             int orCount = TestUtil.nextInt(random(), 1, Math.min(5, dimValues[dim].length));
             drillDowns[dim] = new String[orCount];
             anyMultiValuedDrillDowns |= orCount > 1;
-            for(int i=0;i<orCount;i++) {
+            for (int i = 0; i < orCount; i++) {
               while (true) {
                 String value = dimValues[dim][random().nextInt(dimValues[dim].length)];
-                for(int j=0;j<i;j++) {
+                for (int j = 0; j < i; j++) {
                   if (value.equals(drillDowns[dim][j])) {
                     value = null;
                     break;
@@ -619,7 +685,7 @@ public class TestDrillSideways extends FacetTestCase {
           }
           if (VERBOSE) {
             BytesRef[] values = new BytesRef[drillDowns[dim].length];
-            for(int i=0;i<values.length;i++) {
+            for (int i = 0; i < values.length; i++) {
               values[i] = new BytesRef(drillDowns[dim][i]);
             }
             System.out.println("  dim" + dim + "=" + Arrays.toString(values));
@@ -637,9 +703,9 @@ public class TestDrillSideways extends FacetTestCase {
 
       DrillDownQuery ddq = new DrillDownQuery(config, baseQuery);
 
-      for(int dim=0;dim<numDims;dim++) {
+      for (int dim = 0; dim < numDims; dim++) {
         if (drillDowns[dim] != null) {
-          for(String value : drillDowns[dim]) {
+          for (String value : drillDowns[dim]) {
             ddq.add("dim" + dim, value);
           }
         }
@@ -699,26 +765,25 @@ public class TestDrillSideways extends FacetTestCase {
       // Verify docs are always collected in order.  If we
       // had an AssertingScorer it could catch it when
       // Weight.scoresDocsOutOfOrder lies!:
-      new DrillSideways(s, config, tr).search(ddq,
-                           new SimpleCollector() {
-                             int lastDocID;
-
-                             @Override
-                             public void collect(int doc) {
-                               assert doc > lastDocID;
-                               lastDocID = doc;
-                             }
-
-                             @Override
-                             protected void doSetNextReader(LeafReaderContext context) throws IOException {
-                               lastDocID = -1;
-                             }
-
-                            @Override
-                            public boolean needsScores() {
-                              return false;
-                            }
-                           });
+      getNewDrillSideways(s, config, tr).search(ddq, new SimpleCollector() {
+        int lastDocID;
+
+        @Override
+        public void collect(int doc) {
+          assert doc > lastDocID;
+          lastDocID = doc;
+        }
+
+        @Override
+        protected void doSetNextReader(LeafReaderContext context) throws IOException {
+          lastDocID = -1;
+        }
+
+        @Override
+        public boolean needsScores() {
+          return false;
+        }
+      });
 
       // Also separately verify that DS respects the
       // scoreSubDocsAtOnce method, to ensure that all
@@ -728,12 +793,7 @@ public class TestDrillSideways extends FacetTestCase {
         // drill-down values, because in that case it's
         // easily possible for one of the DD terms to be on
         // a future docID:
-        new DrillSideways(s, config, tr) {
-          @Override
-          protected boolean scoreSubDocsAtOnce() {
-            return true;
-          }
-        }.search(ddq, new AssertingSubDocsAtOnceCollector());
+        getNewDrillSidewaysScoreSubdocsAtOnce(s, config, tr).search(ddq, new AssertingSubDocsAtOnceCollector());
       }
 
       TestFacetResult expected = slowDrillSidewaysSearch(s, docs, contentToken, drillDowns, dimValues, filter);
@@ -741,36 +801,17 @@ public class TestDrillSideways extends FacetTestCase {
       Sort sort = new Sort(new SortField("id", SortField.Type.STRING));
       DrillSideways ds;
       if (doUseDV) {
-        ds = new DrillSideways(s, config, sortedSetDVState);
+        ds = getNewDrillSideways(s, config, sortedSetDVState);
       } else {
-        ds = new DrillSideways(s, config, tr) {
-            @Override
-            protected Facets buildFacetsResult(FacetsCollector drillDowns, FacetsCollector[] drillSideways, String[] drillSidewaysDims) throws IOException {
-              Map<String,Facets> drillSidewaysFacets = new HashMap<>();
-              Facets drillDownFacets = getTaxonomyFacetCounts(taxoReader, config, drillDowns);
-              if (drillSideways != null) {
-                for(int i=0;i<drillSideways.length;i++) {
-                  drillSidewaysFacets.put(drillSidewaysDims[i],
-                                          getTaxonomyFacetCounts(taxoReader, config, drillSideways[i]));
-                }
-              }
-
-              if (drillSidewaysFacets.isEmpty()) {
-                return drillDownFacets;
-              } else {
-                return new MultiFacets(drillSidewaysFacets, drillDownFacets);
-              }
-
-            }
-          };
+        ds = getNewDrillSidewaysBuildFacetsResult(s, config, tr);
       }
 
       // Retrieve all facets:
       DrillSidewaysResult actual = ds.search(ddq, filter, null, numDocs, sort, true, true);
 
       TopDocs hits = s.search(baseQuery, numDocs);
-      Map<String,Float> scores = new HashMap<>();
-      for(ScoreDoc sd : hits.scoreDocs) {
+      Map<String, Float> scores = new HashMap<>();
+      for (ScoreDoc sd : hits.scoreDocs) {
         scores.put(s.doc(sd.doc).get("id"), sd.score);
       }
       if (VERBOSE) {
@@ -781,14 +822,11 @@ public class TestDrillSideways extends FacetTestCase {
       // Make sure drill down doesn't change score:
       Query q = ddq;
       if (filter != null) {
-        q = new BooleanQuery.Builder()
-            .add(q, Occur.MUST)
-            .add(filter, Occur.FILTER)
-            .build();
+        q = new BooleanQuery.Builder().add(q, Occur.MUST).add(filter, Occur.FILTER).build();
       }
       TopDocs ddqHits = s.search(q, numDocs);
       assertEquals(expected.hits.size(), ddqHits.totalHits);
-      for(int i=0;i<expected.hits.size();i++) {
+      for (int i = 0; i < expected.hits.size(); i++) {
         // Score should be IDENTICAL:
         assertEquals(scores.get(expected.hits.get(i).id), ddqHits.scoreDocs[i].score, 0.0f);
       }
@@ -803,7 +841,7 @@ public class TestDrillSideways extends FacetTestCase {
 
     public Counters(String[][] dimValues) {
       counts = new int[dimValues.length][];
-      for(int dim=0;dim<dimValues.length;dim++) {
+      for (int dim = 0; dim < dimValues.length; dim++) {
         counts[dim] = new int[dimValues[dim].length];
       }
     }
@@ -815,7 +853,7 @@ public class TestDrillSideways extends FacetTestCase {
     public void inc(int[] dims, int[] dims2, int onlyDim) {
       assert dims.length == counts.length;
       assert dims2.length == counts.length;
-      for(int dim=0;dim<dims.length;dim++) {
+      for (int dim = 0; dim < dims.length; dim++) {
         if (onlyDim == -1 || dim == onlyDim) {
           if (dims[dim] != -1) {
             counts[dim][dims[dim]]++;
@@ -832,12 +870,14 @@ public class TestDrillSideways extends FacetTestCase {
     List<Doc> hits;
     int[][] counts;
     int[] uniqueCounts;
-    public TestFacetResult() {}
+
+    public TestFacetResult() {
+    }
   }
 
   private int[] getTopNOrds(final int[] counts, final String[] values, int topN) {
     final int[] ids = new int[counts.length];
-    for(int i=0;i<ids.length;i++) {
+    for (int i = 0; i < ids.length; i++) {
       ids[i] = i;
     }
 
@@ -874,7 +914,7 @@ public class TestDrillSideways extends FacetTestCase {
     }
 
     int numSet = topN;
-    for(int i=0;i<topN;i++) {
+    for (int i = 0; i < topN; i++) {
       if (counts[ids[i]] == 0) {
         numSet = i;
         break;
@@ -886,15 +926,14 @@ public class TestDrillSideways extends FacetTestCase {
     return topNIDs;
   }
 
-  private TestFacetResult slowDrillSidewaysSearch(IndexSearcher s, List<Doc> docs,
-                                                        String contentToken, String[][] drillDowns,
-                                                        String[][] dimValues, Query onlyEven) throws Exception {
+  private TestFacetResult slowDrillSidewaysSearch(IndexSearcher s, List<Doc> docs, String contentToken,
+          String[][] drillDowns, String[][] dimValues, Query onlyEven) throws Exception {
     int numDims = dimValues.length;
 
     List<Doc> hits = new ArrayList<>();
     Counters drillDownCounts = new Counters(dimValues);
     Counters[] drillSidewaysCounts = new Counters[dimValues.length];
-    for(int dim=0;dim<numDims;dim++) {
+    for (int dim = 0; dim < numDims; dim++) {
       drillSidewaysCounts[dim] = new Counters(dimValues);
     }
 
@@ -902,7 +941,8 @@ public class TestDrillSideways extends FacetTestCase {
       System.out.println("  compute expected");
     }
 
-    nextDoc: for(Doc doc : docs) {
+    nextDoc:
+    for (Doc doc : docs) {
       if (doc.deleted) {
         continue;
       }
@@ -911,12 +951,12 @@ public class TestDrillSideways extends FacetTestCase {
       }
       if (contentToken == null || doc.contentToken.equals(contentToken)) {
         int failDim = -1;
-        for(int dim=0;dim<numDims;dim++) {
+        for (int dim = 0; dim < numDims; dim++) {
           if (drillDowns[dim] != null) {
             String docValue = doc.dims[dim] == -1 ? null : dimValues[dim][doc.dims[dim]];
             String docValue2 = doc.dims2[dim] == -1 ? null : dimValues[dim][doc.dims2[dim]];
             boolean matches = false;
-            for(String value : drillDowns[dim]) {
+            for (String value : drillDowns[dim]) {
               if (value.equals(docValue) || value.equals(docValue2)) {
                 matches = true;
                 break;
@@ -941,7 +981,7 @@ public class TestDrillSideways extends FacetTestCase {
           // Hit:
           hits.add(doc);
           drillDownCounts.inc(doc.dims, doc.dims2);
-          for(int dim=0;dim<dimValues.length;dim++) {
+          for (int dim = 0; dim < dimValues.length; dim++) {
             drillSidewaysCounts[dim].inc(doc.dims, doc.dims2);
           }
         } else {
@@ -953,8 +993,8 @@ public class TestDrillSideways extends FacetTestCase {
       }
     }
 
-    Map<String,Integer> idToDocID = new HashMap<>();
-    for(int i=0;i<s.getIndexReader().maxDoc();i++) {
+    Map<String, Integer> idToDocID = new HashMap<>();
+    for (int i = 0; i < s.getIndexReader().maxDoc(); i++) {
       idToDocID.put(s.doc(i).get("id"), i);
     }
 
@@ -982,36 +1022,35 @@ public class TestDrillSideways extends FacetTestCase {
     return res;
   }
 
-  void verifyEquals(String[][] dimValues, IndexSearcher s, TestFacetResult expected,
-                    DrillSidewaysResult actual, Map<String,Float> scores, boolean isSortedSetDV) throws Exception {
+  void verifyEquals(String[][] dimValues, IndexSearcher s, TestFacetResult expected, DrillSidewaysResult actual,
+          Map<String, Float> scores, boolean isSortedSetDV) throws Exception {
     if (VERBOSE) {
       System.out.println("  verify totHits=" + expected.hits.size());
     }
     assertEquals(expected.hits.size(), actual.hits.totalHits);
     assertEquals(expected.hits.size(), actual.hits.scoreDocs.length);
-    for(int i=0;i<expected.hits.size();i++) {
+    for (int i = 0; i < expected.hits.size(); i++) {
       if (VERBOSE) {
         System.out.println("    hit " + i + " expected=" + expected.hits.get(i).id);
       }
-      assertEquals(expected.hits.get(i).id,
-                   s.doc(actual.hits.scoreDocs[i].doc).get("id"));
+      assertEquals(expected.hits.get(i).id, s.doc(actual.hits.scoreDocs[i].doc).get("id"));
       // Score should be IDENTICAL:
       assertEquals(scores.get(expected.hits.get(i).id), actual.hits.scoreDocs[i].score, 0.0f);
     }
 
-    for(int dim=0;dim<expected.counts.length;dim++) {
+    for (int dim = 0; dim < expected.counts.length; dim++) {
       int topN = random().nextBoolean() ? dimValues[dim].length : TestUtil.nextInt(random(), 1, dimValues[dim].length);
-      FacetResult fr = actual.facets.getTopChildren(topN, "dim"+dim);
+      FacetResult fr = actual.facets.getTopChildren(topN, "dim" + dim);
       if (VERBOSE) {
         System.out.println("    dim" + dim + " topN=" + topN + " (vs " + dimValues[dim].length + " unique values)");
         System.out.println("      actual");
       }
 
       int idx = 0;
-      Map<String,Integer> actualValues = new HashMap<>();
+      Map<String, Integer> actualValues = new HashMap<>();
 
       if (fr != null) {
-        for(LabelAndValue labelValue : fr.labelValues) {
+        for (LabelAndValue labelValue : fr.labelValues) {
           actualValues.put(labelValue.label, labelValue.value.intValue());
           if (VERBOSE) {
             System.out.println("        " + idx + ": " + new BytesRef(labelValue.label) + ": " + labelValue.value);
@@ -1026,10 +1065,11 @@ public class TestDrillSideways extends FacetTestCase {
         if (VERBOSE) {
           idx = 0;
           System.out.println("      expected (sorted)");
-          for(int i=0;i<topNIDs.length;i++) {
+          for (int i = 0; i < topNIDs.length; i++) {
             int expectedOrd = topNIDs[i];
             String value = dimValues[dim][expectedOrd];
-            System.out.println("        " + idx + ": " + new BytesRef(value) + ": " + expected.counts[dim][expectedOrd]);
+            System.out.println(
+                    "        " + idx + ": " + new BytesRef(value) + ": " + expected.counts[dim][expectedOrd]);
             idx++;
           }
         }
@@ -1042,7 +1082,7 @@ public class TestDrillSideways extends FacetTestCase {
         } else {
           assertEquals(0, topNIDs.length);
         }
-        for(int i=0;i<topNIDs.length;i++) {
+        for (int i = 0; i < topNIDs.length; i++) {
           int expectedOrd = topNIDs[i];
           assertEquals(expected.counts[dim][expectedOrd], fr.labelValues[i].value.intValue());
           if (isSortedSetDV) {
@@ -1056,7 +1096,7 @@ public class TestDrillSideways extends FacetTestCase {
         if (VERBOSE) {
           idx = 0;
           System.out.println("      expected (unsorted)");
-          for(int i=0;i<dimValues[dim].length;i++) {
+          for (int i = 0; i < dimValues[dim].length; i++) {
             String value = dimValues[dim][i];
             if (expected.counts[dim][i] != 0) {
               System.out.println("        " + idx + ": " + new BytesRef(value) + ": " + expected.counts[dim][i]);
@@ -1066,7 +1106,7 @@ public class TestDrillSideways extends FacetTestCase {
         }
 
         int setCount = 0;
-        for(int i=0;i<dimValues[dim].length;i++) {
+        for (int i = 0; i < dimValues[dim].length; i++) {
           String value = dimValues[dim][i];
           if (expected.counts[dim][i] != 0) {
             assertTrue(actualValues.containsKey(value));
@@ -1092,14 +1132,15 @@ public class TestDrillSideways extends FacetTestCase {
 
     // Count "Author"
     FacetsConfig config = new FacetsConfig();
-    DrillSideways ds = new DrillSideways(searcher, config, taxoReader);
+    DrillSideways ds = getNewDrillSideways(searcher, config, taxoReader);
     DrillDownQuery ddq = new DrillDownQuery(config);
     ddq.add("Author", "Lisa");
 
     DrillSidewaysResult r = ds.search(ddq, 10); // this used to fail on IllegalArgEx
     assertEquals(0, r.hits.totalHits);
 
-    r = ds.search(ddq, null, null, 10, new Sort(new SortField("foo", SortField.Type.INT)), false, false); // this used to fail on IllegalArgEx
+    r = ds.search(ddq, null, null, 10, new Sort(new SortField("foo", SortField.Type.INT)), false,
+            false); // this used to fail on IllegalArgEx
     assertEquals(0, r.hits.totalHits);
 
     writer.close();
@@ -1131,7 +1172,7 @@ public class TestDrillSideways extends FacetTestCase {
     // NRT open
     TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoWriter);
 
-    DrillSideways ds = new DrillSideways(searcher, config, taxoReader);
+    DrillSideways ds = getNewDrillSideways(searcher, config, taxoReader);
 
     BooleanQuery.Builder bq = new BooleanQuery.Builder();
     bq.add(new TermQuery(new Term("field", "foo")), BooleanClause.Occur.MUST);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4a002f7/lucene/facet/src/test/org/apache/lucene/facet/TestParallelDrillSideways.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/TestParallelDrillSideways.java b/lucene/facet/src/test/org/apache/lucene/facet/TestParallelDrillSideways.java
new file mode 100644
index 0000000..8f5d20c
--- /dev/null
+++ b/lucene/facet/src/test/org/apache/lucene/facet/TestParallelDrillSideways.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.facet;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import org.apache.lucene.facet.sortedset.SortedSetDocValuesReaderState;
+import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.util.NamedThreadFactory;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+public class TestParallelDrillSideways extends TestDrillSideways {
+
+  private static ExecutorService executor;
+
+  @BeforeClass
+  public static void prepareExecutor() {
+    executor = Executors.newCachedThreadPool(new NamedThreadFactory("TestParallelDrillSideways"));
+  }
+
+  @AfterClass
+  public static void shutdownExecutor() {
+    executor.shutdown();
+    executor = null;
+  }
+
+  protected DrillSideways getNewDrillSideways(IndexSearcher searcher, FacetsConfig config,
+          SortedSetDocValuesReaderState state) {
+    return new DrillSideways(searcher, config, null, state, executor);
+  }
+
+  protected DrillSideways getNewDrillSideways(IndexSearcher searcher, FacetsConfig config, TaxonomyReader taxoReader) {
+    return new DrillSideways(searcher, config, taxoReader, null, executor);
+  }
+
+  protected DrillSideways getNewDrillSidewaysScoreSubdocsAtOnce(IndexSearcher searcher, FacetsConfig config,
+          TaxonomyReader taxoReader) {
+    return new DrillSideways(searcher, config, taxoReader, null, executor) {
+      @Override
+      protected boolean scoreSubDocsAtOnce() {
+        return true;
+      }
+    };
+  }
+
+  protected DrillSideways getNewDrillSidewaysBuildFacetsResult(IndexSearcher searcher, FacetsConfig config,
+          TaxonomyReader taxoReader) {
+    return new DrillSideways(searcher, config, taxoReader, null, executor) {
+      @Override
+      protected Facets buildFacetsResult(FacetsCollector drillDowns, FacetsCollector[] drillSideways,
+              String[] drillSidewaysDims) throws IOException {
+        Map<String, Facets> drillSidewaysFacets = new HashMap<>();
+        Facets drillDownFacets = getTaxonomyFacetCounts(taxoReader, config, drillDowns);
+        if (drillSideways != null) {
+          for (int i = 0; i < drillSideways.length; i++) {
+            drillSidewaysFacets.put(drillSidewaysDims[i], getTaxonomyFacetCounts(taxoReader, config, drillSideways[i]));
+          }
+        }
+
+        if (drillSidewaysFacets.isEmpty()) {
+          return drillDownFacets;
+        } else {
+          return new MultiFacets(drillSidewaysFacets, drillDownFacets);
+        }
+
+      }
+    };
+  }
+
+}


[11/50] lucene-solr:jira/solr-8593: tests: raise commit time to avoid false fails

Posted by kr...@apache.org.
tests: raise commit time to avoid false fails


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

Branch: refs/heads/jira/solr-8593
Commit: 5f55ae0b73ec546132f7188490065798bba0ffad
Parents: 197590a
Author: markrmiller <ma...@apache.org>
Authored: Thu Dec 29 05:53:51 2016 -0500
Committer: markrmiller <ma...@apache.org>
Committed: Thu Dec 29 05:53:51 2016 -0500

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


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5f55ae0b/solr/core/src/test/org/apache/solr/update/AutoCommitTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/AutoCommitTest.java b/solr/core/src/test/org/apache/solr/update/AutoCommitTest.java
index f68e563..cb030e4 100644
--- a/solr/core/src/test/org/apache/solr/update/AutoCommitTest.java
+++ b/solr/core/src/test/org/apache/solr/update/AutoCommitTest.java
@@ -239,7 +239,7 @@ public class AutoCommitTest extends AbstractSolrTestCase {
     CommitTracker tracker = updater.softCommitTracker;
     // too low of a number can cause a slow host to commit before the test code checks that it
     // isn't there... causing a failure at "shouldn't find any"
-    tracker.setTimeUpperBound(1000);
+    tracker.setTimeUpperBound(1500);
     tracker.setDocsUpperBound(-1);
     // updater.commitCallbacks.add(trigger);
     


[39/50] lucene-solr:jira/solr-8593: Merge remote-tracking branch 'origin/master'

Posted by kr...@apache.org.
Merge remote-tracking branch 'origin/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/33304f20
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/33304f20
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/33304f20

Branch: refs/heads/jira/solr-8593
Commit: 33304f202c88003b4bb2ac2d02307441e5a54807
Parents: 3988532 0999f67
Author: Noble Paul <no...@apache.org>
Authored: Tue Jan 3 17:53:34 2017 +1030
Committer: Noble Paul <no...@apache.org>
Committed: Tue Jan 3 17:53:34 2017 +1030

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   6 +-
 .../org/apache/solr/handler/StreamHandler.java  |   2 +-
 .../solr/spelling/DirectSolrSpellChecker.java   |  18 ++-
 .../reporters/SolrGangliaReporterTest.java      |   3 +-
 .../reporters/SolrGraphiteReporterTest.java     |   3 +-
 .../client/solrj/io/stream/PriorityStream.java  | 161 +++++++++++++++++++
 .../client/solrj/io/stream/SchedulerStream.java | 161 -------------------
 .../solrj/io/stream/StreamExpressionTest.java   |  20 +--
 8 files changed, 191 insertions(+), 183 deletions(-)
----------------------------------------------------------------------



[14/50] lucene-solr:jira/solr-8593: tests: speed up non nightly run

Posted by kr...@apache.org.
tests: speed up non nightly run


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

Branch: refs/heads/jira/solr-8593
Commit: fa959ad25d2460ebb41fae6bcf496a5ce785e989
Parents: c58eaa1
Author: markrmiller <ma...@apache.org>
Authored: Thu Dec 29 06:42:14 2016 -0500
Committer: markrmiller <ma...@apache.org>
Committed: Thu Dec 29 06:42:14 2016 -0500

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


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fa959ad2/solr/core/src/test/org/apache/solr/cloud/TestStressLiveNodes.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestStressLiveNodes.java b/solr/core/src/test/org/apache/solr/cloud/TestStressLiveNodes.java
index 80b717e..52ab3a3 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestStressLiveNodes.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestStressLiveNodes.java
@@ -132,7 +132,7 @@ public class TestStressLiveNodes extends SolrCloudTestCase {
   public void testStress() throws Exception {
 
     // do many iters, so we have "bursts" of adding nodes that we then check
-    final int numIters = atLeast(1000);
+    final int numIters = atLeast(TEST_NIGHTLY ? 1000 : 100);
     for (int iter = 0; iter < numIters; iter++) {
 
       // sanity check that ZK says there is in fact 1 live node


[24/50] lucene-solr:jira/solr-8593: SOLR-9900: fix false positives on range queries with ReversedWildcardFilterFactory

Posted by kr...@apache.org.
SOLR-9900: fix false positives on range queries with ReversedWildcardFilterFactory


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

Branch: refs/heads/jira/solr-8593
Commit: 5d042d3a49dfcf654b8bf8a96521d5404bfd3a7b
Parents: 6b00ee5
Author: Mikhail Khludnev <mk...@apache.org>
Authored: Fri Dec 30 00:42:51 2016 +0300
Committer: Mikhail Khludnev <mk...@apache.org>
Committed: Fri Dec 30 00:43:29 2016 +0300

----------------------------------------------------------------------
 solr/CHANGES.txt                                      |  2 ++
 .../org/apache/solr/parser/SolrQueryParserBase.java   | 14 ++++++++++++++
 .../analysis/TestReversedWildcardFilterFactory.java   | 12 ++++++++++++
 3 files changed, 28 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5d042d3a/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index b509e23..7f83de0 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -298,6 +298,8 @@ Bug Fixes
 
 * SOLR-9901: Implement move in HdfsDirectoryFactory. (Mark Miller)
 
+* SOLR-9900: fix false positives on range queries with ReversedWildcardFilterFactory (Yonik Seeley via Mikhail Khludnev)
+
 Other Changes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5d042d3a/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java b/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java
index 168bd49..f54e9e9 100644
--- a/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java
+++ b/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.reverse.ReverseStringFilter;
 import org.apache.lucene.analysis.util.TokenFilterFactory;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.AutomatonQuery;
@@ -894,6 +895,19 @@ public abstract class SolrQueryParserBase extends QueryBuilder {
   protected Query getRangeQuery(String field, String part1, String part2, boolean startInclusive, boolean endInclusive) throws SyntaxError {
     checkNullField(field);
     SchemaField sf = schema.getField(field);
+
+    if (part1 == null) {
+      ReversedWildcardFilterFactory factory = getReversedWildcardFilterFactory(sf.getType());
+      if (factory != null) {
+        // There will be reversed tokens starting with u0001 that we want to exclude, so
+        // lets start at u0002 inclusive instead.
+        char[] buf = new char[1];
+        buf[0] = ReverseStringFilter.START_OF_HEADING_MARKER + 1;
+        part1 = new String(buf);
+        startInclusive = true;
+      }
+    }
+
     return sf.getType().getRangeQuery(parser, sf, part1, part2, startInclusive, endInclusive);
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5d042d3a/solr/core/src/test/org/apache/solr/analysis/TestReversedWildcardFilterFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/analysis/TestReversedWildcardFilterFactory.java b/solr/core/src/test/org/apache/solr/analysis/TestReversedWildcardFilterFactory.java
index 3ccc352..f7a49ac 100644
--- a/solr/core/src/test/org/apache/solr/analysis/TestReversedWildcardFilterFactory.java
+++ b/solr/core/src/test/org/apache/solr/analysis/TestReversedWildcardFilterFactory.java
@@ -182,6 +182,18 @@ public class TestReversedWildcardFilterFactory extends SolrTestCaseJ4 {
     assertQ("false positive",
         req("+id:1 +one:*zemog*"),
         "//result[@numFound=0]");
+    
+    assertQ("no reverse, no false positive",
+        req("q", "+id:1 +three:[* TO a]", 
+            "debugQuery", "true"),
+        "//result[@numFound=0]");
+    {
+      String reverseField = random().nextBoolean() ? "one":"two";
+      assertQ("false positive",
+          req("q", "+id:1 +"+reverseField+":[* TO a]", 
+              "debugQuery", "true"),
+          "//result[@numFound=0]");
+    }
     assertQ("false positive",
         req("+id:1 +two:*zemog*"),
         "//result[@numFound=0]");


[38/50] lucene-solr:jira/solr-8593: SOLR-9906-Use better check to validate if node recovered via PeerSync or Replication

Posted by kr...@apache.org.
SOLR-9906-Use better check to validate if node recovered via PeerSync or Replication


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

Branch: refs/heads/jira/solr-8593
Commit: 3988532d26a50b1f3cf51e1d0009a0754cfd6b57
Parents: 832d02b
Author: Noble Paul <no...@apache.org>
Authored: Tue Jan 3 17:52:08 2017 +1030
Committer: Noble Paul <no...@apache.org>
Committed: Tue Jan 3 17:52:08 2017 +1030

----------------------------------------------------------------------
 .../src/java/org/apache/solr/util/TimeOut.java  | 13 +++-
 .../cloud/LeaderFailureAfterFreshStartTest.java | 68 +++++++++++---------
 .../solr/cloud/PeerSyncReplicationTest.java     | 55 +++++++---------
 .../solr/cloud/AbstractDistribZkTestBase.java   | 32 ++++-----
 4 files changed, 89 insertions(+), 79 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3988532d/solr/core/src/java/org/apache/solr/util/TimeOut.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/TimeOut.java b/solr/core/src/java/org/apache/solr/util/TimeOut.java
index f823b7e..fd91045 100644
--- a/solr/core/src/java/org/apache/solr/util/TimeOut.java
+++ b/solr/core/src/java/org/apache/solr/util/TimeOut.java
@@ -18,12 +18,15 @@ package org.apache.solr.util;
 
 import java.util.concurrent.TimeUnit;
 
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
 public class TimeOut {
 
-  private final long timeoutAt;
+  private final long timeoutAt, startTime;
 
   public TimeOut(long interval, TimeUnit unit) {
-    this.timeoutAt = System.nanoTime() + TimeUnit.NANOSECONDS.convert(interval, unit);
+    startTime = System.nanoTime();
+    this.timeoutAt = startTime + NANOSECONDS.convert(interval, unit);
   }
 
   public boolean hasTimedOut() {
@@ -31,6 +34,10 @@ public class TimeOut {
   }
 
   public long timeLeft(TimeUnit unit) {
-    return unit.convert(timeoutAt - System.nanoTime(), TimeUnit.NANOSECONDS);
+    return unit.convert(timeoutAt - System.nanoTime(), NANOSECONDS);
+  }
+
+  public long timeElapsed(TimeUnit unit) {
+    return unit.convert(System.nanoTime() - startTime, NANOSECONDS);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3988532d/solr/core/src/test/org/apache/solr/cloud/LeaderFailureAfterFreshStartTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/LeaderFailureAfterFreshStartTest.java b/solr/core/src/test/org/apache/solr/cloud/LeaderFailureAfterFreshStartTest.java
index ef21386..77dd6b6 100644
--- a/solr/core/src/test/org/apache/solr/cloud/LeaderFailureAfterFreshStartTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/LeaderFailureAfterFreshStartTest.java
@@ -19,6 +19,8 @@ package org.apache.solr.cloud;
 
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
+import java.nio.file.Files;
+import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashSet;
@@ -26,6 +28,7 @@ import java.util.List;
 import java.util.Set;
 import java.util.stream.Collectors;
 
+import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.commons.lang.RandomStringUtils;
 import org.apache.lucene.util.LuceneTestCase.Slow;
 import org.apache.solr.client.solrj.SolrServerException;
@@ -38,12 +41,13 @@ import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.solr.handler.ReplicationHandler;
+import org.apache.solr.util.TimeOut;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static java.util.Collections.singletonList;
+import static java.util.concurrent.TimeUnit.SECONDS;
 
 /**
  * 
@@ -127,52 +131,53 @@ public class LeaderFailureAfterFreshStartTest extends AbstractFullDistribZkTestB
       waitForThingsToLevelOut(30);
 
       checkShardConsistency(false, true);
+      
+      // bring down the other node and index a few docs; so the leader and other node segments diverge
+      forceNodeFailures(singletonList(secondNode));
+      for (int i = 0; i < 10; i++) {
+        indexDoc(id, docId, i1, 50, tlong, 50, t1,
+            "document number " + docId++);
+        if(i % 2 == 0) {
+          commit();
+        }
+      }
+      commit();
+      restartNodes(singletonList(secondNode));
 
       // start the freshNode 
-      ChaosMonkey.start(freshNode.jetty);
-      nodesDown.remove(freshNode);
-
-      waitTillNodesActive();
-      waitForThingsToLevelOut(30);
-      
-      //TODO check how to see if fresh node went into recovery (may be check count for replication handler on new leader) 
-      
-      long numRequestsBefore = (Long) secondNode.jetty
-          .getCoreContainer()
-          .getCores()
-          .iterator()
-          .next()
-          .getRequestHandler(ReplicationHandler.PATH)
-          .getStatistics().get("requests");
+      restartNodes(singletonList(freshNode));
       
+      String replicationProperties = (String) freshNode.jetty.getSolrHome() + "/cores/" +  DEFAULT_TEST_COLLECTION_NAME + "/data/replication.properties";
+      String md5 = DigestUtils.md5Hex(Files.readAllBytes(Paths.get(replicationProperties)));
+        
       // shutdown the original leader
       log.info("Now shutting down initial leader");
       forceNodeFailures(singletonList(initialLeaderJetty));
-      waitForNewLeader(cloudClient, "shard1", (Replica)initialLeaderJetty.client.info  , 15);
+      waitForNewLeader(cloudClient, "shard1", (Replica)initialLeaderJetty.client.info  , new TimeOut(15, SECONDS));
+      waitTillNodesActive();
       log.info("Updating mappings from zk");
       updateMappingsFromZk(jettys, clients, true);
-      
-      long numRequestsAfter = (Long) secondNode.jetty
-          .getCoreContainer()
-          .getCores()
-          .iterator()
-          .next()
-          .getRequestHandler(ReplicationHandler.PATH)
-          .getStatistics().get("requests");
-
-      assertEquals("Node went into replication", numRequestsBefore, numRequestsAfter);
+      assertEquals("Node went into replication", md5, DigestUtils.md5Hex(Files.readAllBytes(Paths.get(replicationProperties))));
       
       success = true;
     } finally {
       System.clearProperty("solr.disableFingerprint");
     }
   }
+  
+  private void restartNodes(List<CloudJettyRunner> nodesToRestart) throws Exception {
+    for (CloudJettyRunner node : nodesToRestart) {
+      chaosMonkey.start(node.jetty);
+      nodesDown.remove(node);
+    }
+    waitTillNodesActive();
+    checkShardConsistency(false, true);
+  }
 
 
   private void forceNodeFailures(List<CloudJettyRunner> replicasToShutDown) throws Exception {
     for (CloudJettyRunner replicaToShutDown : replicasToShutDown) {
       chaosMonkey.killJetty(replicaToShutDown);
-      waitForNoShardInconsistency();
     }
 
     int totalDown = 0;
@@ -205,8 +210,13 @@ public class LeaderFailureAfterFreshStartTest extends AbstractFullDistribZkTestB
       Collection<Replica> replicas = slice.getReplicas();
       boolean allActive = true;
 
+      Collection<String> nodesDownNames = nodesDown.stream()
+          .map(n -> n.coreNodeName)
+          .collect(Collectors.toList());
+      
       Collection<Replica> replicasToCheck = null;
-      replicasToCheck = replicas.stream().filter(r -> nodesDown.contains(r.getName()))
+      replicasToCheck = replicas.stream()
+          .filter(r -> !nodesDownNames.contains(r.getName()))
           .collect(Collectors.toList());
 
       for (Replica replica : replicasToCheck) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3988532d/solr/core/src/test/org/apache/solr/cloud/PeerSyncReplicationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/PeerSyncReplicationTest.java b/solr/core/src/test/org/apache/solr/cloud/PeerSyncReplicationTest.java
index e00ea3c..4084ad7 100644
--- a/solr/core/src/test/org/apache/solr/cloud/PeerSyncReplicationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/PeerSyncReplicationTest.java
@@ -20,6 +20,8 @@ package org.apache.solr.cloud;
 
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
+import java.nio.file.Files;
+import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -41,15 +43,16 @@ import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.solr.handler.ReplicationHandler;
+import org.apache.solr.util.TimeOut;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static java.util.Collections.singletonList;
+import static java.util.concurrent.TimeUnit.SECONDS;
 
 /**
- * Test sync peer sync when a node restarts and documents are indexed when node was down.
+ * Test PeerSync when a node restarts and documents are indexed when node was down.
  *
  * This test is modeled after SyncSliceTest
  */
@@ -149,12 +152,12 @@ public class PeerSyncReplicationTest extends AbstractFullDistribZkTestBase {
       log.info("Now shutting down initial leader");
       forceNodeFailures(singletonList(initialLeaderJetty));
       log.info("Updating mappings from zk");
-      waitForNewLeader(cloudClient, "shard1", (Replica) initialLeaderJetty.client.info, 15);
+      waitForNewLeader(cloudClient, "shard1", (Replica) initialLeaderJetty.client.info, new TimeOut(15, SECONDS));
       updateMappingsFromZk(jettys, clients, true);
       assertEquals("PeerSynced node did not become leader", nodePeerSynced, shardToLeaderJetty.get("shard1"));
 
       // bring up node that was down all along, and let it PeerSync from the node that was forced to PeerSynce  
-      bringUpDeadNodeAndEnsureNoReplication(shardToLeaderJetty.get("shard1"), neverLeader, false);
+      bringUpDeadNodeAndEnsureNoReplication(neverLeader, false);
       waitTillNodesActive();
 
       checkShardConsistency(false, true);
@@ -199,7 +202,6 @@ public class PeerSyncReplicationTest extends AbstractFullDistribZkTestBase {
   private void forceNodeFailures(List<CloudJettyRunner> replicasToShutDown) throws Exception {
     for (CloudJettyRunner replicaToShutDown : replicasToShutDown) {
       chaosMonkey.killJetty(replicaToShutDown);
-      waitForNoShardInconsistency();
     }
 
     int totalDown = 0;
@@ -218,8 +220,6 @@ public class PeerSyncReplicationTest extends AbstractFullDistribZkTestBase {
     assertEquals(getShardCount() - totalDown, jetties.size());
 
     nodesDown.addAll(replicasToShutDown);
-
-    Thread.sleep(3000);
   }
   
   
@@ -241,26 +241,17 @@ public class PeerSyncReplicationTest extends AbstractFullDistribZkTestBase {
         "document number " + docId++);
     commit();
 
-    bringUpDeadNodeAndEnsureNoReplication(leaderJetty, replicaToShutDown, disableFingerprint);
+    bringUpDeadNodeAndEnsureNoReplication(replicaToShutDown, disableFingerprint);
 
     return replicaToShutDown;
   }
-  
-  
 
-  private void bringUpDeadNodeAndEnsureNoReplication(CloudJettyRunner leaderJetty, CloudJettyRunner nodeToBringUp,
-      boolean disableFingerprint) throws Exception {
+
+  private void bringUpDeadNodeAndEnsureNoReplication(CloudJettyRunner nodeToBringUp, boolean disableFingerprint)
+      throws Exception {
     // disable fingerprint check if needed
     System.setProperty("solr.disableFingerprint", String.valueOf(disableFingerprint));
 
-    long numRequestsBefore = (Long) leaderJetty.jetty
-        .getCoreContainer()
-        .getCores()
-        .iterator()
-        .next()
-        .getRequestHandler(ReplicationHandler.PATH)
-        .getStatistics().get("requests");
-
     indexInBackground(50);
     
     // bring back dead node and ensure it recovers
@@ -279,15 +270,9 @@ public class PeerSyncReplicationTest extends AbstractFullDistribZkTestBase {
     long cloudClientDocs = cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound();
     assertEquals(docId, cloudClientDocs);
 
-    long numRequestsAfter = (Long) leaderJetty.jetty
-        .getCoreContainer()
-        .getCores()
-        .iterator()
-        .next()
-        .getRequestHandler(ReplicationHandler.PATH)
-        .getStatistics().get("requests");
-
-    assertEquals("PeerSync failed. Had to fail back to replication", numRequestsBefore, numRequestsAfter);
+    // if there was no replication, we should not have replication.properties file
+    String replicationProperties = nodeToBringUp.jetty.getSolrHome() + "/cores/" + DEFAULT_TEST_COLLECTION_NAME + "/data/replication.properties";
+    assertTrue("PeerSync failed. Had to fail back to replication", Files.notExists(Paths.get(replicationProperties)));
   }
 
   
@@ -302,9 +287,15 @@ public class PeerSyncReplicationTest extends AbstractFullDistribZkTestBase {
       Collection<Replica> replicas = slice.getReplicas();
       boolean allActive = true;
 
-      Collection<Replica> replicasToCheck = null;
-      replicasToCheck = replicas.stream().filter(r -> nodesDown.contains(r.getName()))
-            .collect(Collectors.toList());
+      Collection<String> nodesDownNames =
+          nodesDown.stream()
+              .map(n -> n.coreNodeName)
+              .collect(Collectors.toList());
+
+      Collection<Replica> replicasToCheck =
+          replicas.stream()
+              .filter(r -> !nodesDownNames.contains(r.getName()))
+              .collect(Collectors.toList());
 
       for (Replica replica : replicasToCheck) {
         if (!clusterState.liveNodesContain(replica.getNodeName()) || replica.getState() != Replica.State.ACTIVE) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3988532d/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java
index d04d996..7f991a4 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java
@@ -19,6 +19,7 @@ package org.apache.solr.cloud;
 import java.io.File;
 import java.lang.invoke.MethodHandles;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.io.FileUtils;
@@ -29,16 +30,20 @@ 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.Slice.State;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.cloud.Slice.State;
 import org.apache.solr.core.Diagnostics;
 import org.apache.solr.core.MockDirectoryFactory;
+import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
 import org.junit.BeforeClass;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+
 public abstract class AbstractDistribZkTestBase extends BaseDistributedSearchTestCase {
   
   private static final String REMOVE_VERSION_FIELD = "remove.version.field";
@@ -226,31 +231,28 @@ public abstract class AbstractDistribZkTestBase extends BaseDistributedSearchTes
     log.info("Collection has disappeared - collection: " + collection);
   }
   
-  static void waitForNewLeader(CloudSolrClient cloudClient, String shardName, Replica oldLeader, int maxWaitInSecs)
+  static void waitForNewLeader(CloudSolrClient cloudClient, String shardName, Replica oldLeader, TimeOut timeOut)
       throws Exception {
-    log.info("Will wait for a node to become leader for {} secs", maxWaitInSecs);
-    boolean waitForLeader = true;
-    int i = 0;
+    log.info("Will wait for a node to become leader for {} secs", timeOut.timeLeft(SECONDS));
     ZkStateReader zkStateReader = cloudClient.getZkStateReader();
     zkStateReader.forceUpdateCollection(DEFAULT_COLLECTION);
-    
-    while(waitForLeader) {
+
+    for (; ; ) {
       ClusterState clusterState = zkStateReader.getClusterState();
       DocCollection coll = clusterState.getCollection("collection1");
       Slice slice = coll.getSlice(shardName);
-      if(slice.getLeader() != oldLeader && slice.getState() == State.ACTIVE) {
-        log.info("New leader got elected in {} secs", i);
+      if (slice.getLeader() != null && !slice.getLeader().equals(oldLeader) && slice.getState() == State.ACTIVE) {
+        log.info("Old leader {}, new leader. New leader got elected in {} ms", oldLeader, slice.getLeader(),timeOut.timeElapsed(MILLISECONDS) );
         break;
       }
-      
-      if(i == maxWaitInSecs) {
+
+      if (timeOut.hasTimedOut()) {
         Diagnostics.logThreadDumps("Could not find new leader in specified timeout");
         zkStateReader.getZkClient().printLayoutToStdOut();
-        fail("Could not find new leader even after waiting for " + maxWaitInSecs + "secs");
+        fail("Could not find new leader even after waiting for " + timeOut.timeElapsed(MILLISECONDS) + "ms");
       }
-      
-      i++;
-      Thread.sleep(1000);
+
+      Thread.sleep(100);
     }
   }
 


[13/50] lucene-solr:jira/solr-8593: tests: speed up very slow test

Posted by kr...@apache.org.
tests: speed up very slow 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/c58eaa1a
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/c58eaa1a
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/c58eaa1a

Branch: refs/heads/jira/solr-8593
Commit: c58eaa1a49bf518f3b0f70701ffd31f0cca79c17
Parents: b4de628
Author: markrmiller <ma...@apache.org>
Authored: Wed Dec 28 08:36:08 2016 -0500
Committer: markrmiller <ma...@apache.org>
Committed: Thu Dec 29 06:22:58 2016 -0500

----------------------------------------------------------------------
 .../apache/solr/cloud/CdcrBootstrapTest.java    | 25 ++++++++++----------
 1 file changed, 12 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c58eaa1a/solr/core/src/test/org/apache/solr/cloud/CdcrBootstrapTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/CdcrBootstrapTest.java b/solr/core/src/test/org/apache/solr/cloud/CdcrBootstrapTest.java
index aca5e0b..6959bd8 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CdcrBootstrapTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CdcrBootstrapTest.java
@@ -76,11 +76,11 @@ public class CdcrBootstrapTest extends SolrTestCaseJ4 {
             .withProperty("solr.directoryFactory", "solr.StandardDirectoryFactory")
             .process(source.getSolrClient());
 
-        // index 10000 docs with a hard commit every 1000 documents
         CloudSolrClient sourceSolrClient = source.getSolrClient();
         sourceSolrClient.setDefaultCollection("cdcr-source");
+        int docs = (TEST_NIGHTLY ? 100 : 10);
         int numDocs = 0;
-        for (int k = 0; k < 100; k++) {
+        for (int k = 0; k < docs; k++) {
           UpdateRequest req = new UpdateRequest();
           for (; numDocs < (k + 1) * 100; numDocs++) {
             SolrInputDocument doc = new SolrInputDocument();
@@ -89,7 +89,7 @@ public class CdcrBootstrapTest extends SolrTestCaseJ4 {
             req.add(doc);
           }
           req.setAction(AbstractUpdateRequest.ACTION.COMMIT, true, true);
-          System.out.println("Adding 100 docs with commit=true, numDocs=" + numDocs);
+          System.out.println("Adding " + docs + " docs with commit=true, numDocs=" + numDocs);
           req.process(sourceSolrClient);
         }
 
@@ -170,11 +170,11 @@ public class CdcrBootstrapTest extends SolrTestCaseJ4 {
             .withProperty("solr.directoryFactory", "solr.StandardDirectoryFactory")
             .process(source.getSolrClient());
 
-        // index 10000 docs with a hard commit every 1000 documents
         CloudSolrClient sourceSolrClient = source.getSolrClient();
         sourceSolrClient.setDefaultCollection("cdcr-source");
+        int docs = (TEST_NIGHTLY ? 100 : 10);
         int numDocs = 0;
-        for (int k = 0; k < 100; k++) {
+        for (int k = 0; k < docs; k++) {
           UpdateRequest req = new UpdateRequest();
           for (; numDocs < (k + 1) * 100; numDocs++) {
             SolrInputDocument doc = new SolrInputDocument();
@@ -183,7 +183,7 @@ public class CdcrBootstrapTest extends SolrTestCaseJ4 {
             req.add(doc);
           }
           req.setAction(AbstractUpdateRequest.ACTION.COMMIT, true, true);
-          System.out.println("Adding 100 docs with commit=true, numDocs=" + numDocs);
+          System.out.println("Adding " + docs + " docs with commit=true, numDocs=" + numDocs);
           req.process(sourceSolrClient);
         }
 
@@ -209,7 +209,7 @@ public class CdcrBootstrapTest extends SolrTestCaseJ4 {
         cdcrDisableBuffer(sourceSolrClient);
 
         int c = 0;
-        for (int k = 0; k < 100; k++) {
+        for (int k = 0; k < 10; k++) {
           UpdateRequest req = new UpdateRequest();
           for (; c < (k + 1) * 100; c++, numDocs++) {
             SolrInputDocument doc = new SolrInputDocument();
@@ -256,11 +256,11 @@ public class CdcrBootstrapTest extends SolrTestCaseJ4 {
             .withProperty("solr.directoryFactory", "solr.StandardDirectoryFactory")
             .process(source.getSolrClient());
 
-        // index 10000 docs with a hard commit every 1000 documents
         CloudSolrClient sourceSolrClient = source.getSolrClient();
         sourceSolrClient.setDefaultCollection("cdcr-source");
+        int docs = (TEST_NIGHTLY ? 100 : 10);
         int numDocs = 0;
-        for (int k = 0; k < 100; k++) {
+        for (int k = 0; k < docs; k++) {
           UpdateRequest req = new UpdateRequest();
           for (; numDocs < (k + 1) * 100; numDocs++) {
             SolrInputDocument doc = new SolrInputDocument();
@@ -269,7 +269,7 @@ public class CdcrBootstrapTest extends SolrTestCaseJ4 {
             req.add(doc);
           }
           req.setAction(AbstractUpdateRequest.ACTION.COMMIT, true, true);
-          System.out.println("Adding 100 docs with commit=true, numDocs=" + numDocs);
+          System.out.println("Adding " + docs + " docs with commit=true, numDocs=" + numDocs);
           req.process(sourceSolrClient);
         }
 
@@ -286,9 +286,8 @@ public class CdcrBootstrapTest extends SolrTestCaseJ4 {
 
         cdcrStart(targetSolrClient);
         cdcrStart(sourceSolrClient);
-
         int c = 0;
-        for (int k = 0; k < 100; k++) {
+        for (int k = 0; k < docs; k++) {
           UpdateRequest req = new UpdateRequest();
           for (; c < (k + 1) * 100; c++, numDocs++) {
             SolrInputDocument doc = new SolrInputDocument();
@@ -297,7 +296,7 @@ public class CdcrBootstrapTest extends SolrTestCaseJ4 {
             req.add(doc);
           }
           req.setAction(AbstractUpdateRequest.ACTION.COMMIT, true, true);
-          System.out.println("Adding 100 docs with commit=true, numDocs=" + numDocs);
+          System.out.println("Adding " + docs + " docs with commit=true, numDocs=" + numDocs);
           req.process(sourceSolrClient);
         }
 


[19/50] lucene-solr:jira/solr-8593: SOLR-9905: Add NullStream to isolate the performance of the ExportWriter

Posted by kr...@apache.org.
SOLR-9905: Add NullStream to isolate the performance of the ExportWriter


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

Branch: refs/heads/jira/solr-8593
Commit: 7dcb557ab73da7fb7af0e8f698895e28dde4bbca
Parents: db9190d
Author: Joel Bernstein <jb...@apache.org>
Authored: Thu Dec 29 13:46:04 2016 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Thu Dec 29 14:39:16 2016 -0500

----------------------------------------------------------------------
 .../org/apache/solr/handler/StreamHandler.java  |   2 +-
 .../solr/client/solrj/io/stream/NullStream.java | 155 +++++++++++++++++++
 .../solrj/io/stream/StreamExpressionTest.java   |  65 ++++++++
 3 files changed, 221 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7dcb557a/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 c6f3c62..13ce636 100644
--- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
@@ -139,7 +139,7 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
       .withFunctionName("classify", ClassifyStream.class)
       .withFunctionName("fetch", FetchStream.class)
       .withFunctionName("executor", ExecutorStream.class)
-
+      .withFunctionName("null", NullStream.class)
       // metrics
       .withFunctionName("min", MinMetric.class)
       .withFunctionName("max", MaxMetric.class)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7dcb557a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/NullStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/NullStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/NullStream.java
new file mode 100644
index 0000000..bef3b1c
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/NullStream.java
@@ -0,0 +1,155 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Date;
+
+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;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType;
+import org.apache.solr.client.solrj.io.stream.expr.Expressible;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExplanation;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+
+/**
+ *  The NullStream Iterates over a TupleStream and eats the tuples. It returns the tuple count in the EOF tuple.
+ *  Because the NullStreaam eats all the Tuples it see's it can be used as a simple tool for performance analysis of
+ *  underlying streams.
+ **/
+
+public class NullStream extends TupleStream implements Expressible {
+
+  private static final long serialVersionUID = 1;
+
+  private TupleStream stream;
+  private long count;
+  private long start;
+  private Tuple eof;
+
+  public NullStream(TupleStream tupleStream) throws IOException {
+    init(tupleStream);
+  }
+
+  public NullStream(StreamExpression expression, StreamFactory factory) throws IOException {
+    // grab all parameters out
+    List<StreamExpression> streamExpressions = factory.getExpressionOperandsRepresentingTypes(expression, Expressible.class, TupleStream.class);
+    TupleStream stream = factory.constructStream(streamExpressions.get(0));
+
+    init(stream);
+  }
+
+  private void init(TupleStream tupleStream) throws IOException{
+    this.stream = tupleStream;
+  }
+
+  @Override
+  public StreamExpression toExpression(StreamFactory factory) throws IOException{
+    return toExpression(factory, true);
+  }
+
+  private StreamExpression toExpression(StreamFactory factory, boolean includeStreams) throws IOException {
+    // function name
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
+
+    if(includeStreams){
+      // stream
+      if(stream instanceof Expressible){
+        expression.addParameter(((Expressible)stream).toExpression(factory));
+      }
+      else{
+        throw new IOException("This RankStream contains a non-expressible TupleStream - it cannot be converted to an expression");
+      }
+    }
+    else{
+      expression.addParameter("<stream>");
+    }
+
+    return expression;
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+
+    return new StreamExplanation(getStreamNodeId().toString())
+        .withChildren(new Explanation[]{
+            stream.toExplanation(factory)
+        })
+        .withFunctionName(factory.getFunctionName(this.getClass()))
+        .withImplementingClass(this.getClass().getName())
+        .withExpressionType(ExpressionType.STREAM_DECORATOR)
+        .withExpression(toExpression(factory, false).toString());
+  }
+
+  public void setStreamContext(StreamContext context) {
+    this.stream.setStreamContext(context);
+  }
+
+  public List<TupleStream> children() {
+    List<TupleStream> l =  new ArrayList<TupleStream>();
+    l.add(stream);
+    return l;
+  }
+
+  public void open() throws IOException {
+    start = new Date().getTime();
+    count = 0;
+    stream.open();
+  }
+
+  public void close() throws IOException {
+    stream.close();
+  }
+
+  public Tuple read() throws IOException {
+
+    if(eof != null) {
+      return eof;
+    }
+
+    while(true) {
+      Tuple tuple  = stream.read();
+      if(tuple.EOF) {
+        eof = tuple;
+        long end = new Date().getTime();
+        Tuple t = new Tuple(new HashMap());
+        t.put("nullCount", count);
+        t.put("timer", end-start);
+        return t;
+      } else {
+        ++count;
+      }
+    }
+  }
+
+  /** Return the stream sort - ie, the order in which records are returned */
+  public StreamComparator getStreamSort(){
+    return stream.getStreamSort();
+  }
+
+  public int getCost() {
+    return 0;
+  }
+
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7dcb557a/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 7d48c0e..0c9d5b3 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
@@ -373,6 +373,71 @@ public class StreamExpressionTest extends SolrCloudTestCase {
 
   }
 
+
+  @Test
+  public void testNullStream() throws Exception {
+
+    new UpdateRequest()
+        .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0")
+        .add(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0")
+        .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3")
+        .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4")
+        .add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1")
+        .add(id, "5", "a_s", "hello1", "a_i", "1", "a_f", "2")
+        .commit(cluster.getSolrClient(), COLLECTIONORALIAS);
+
+    StreamExpression expression;
+    TupleStream stream;
+    List<Tuple> tuples;
+
+    StreamFactory factory = new StreamFactory()
+        .withCollectionZkHost(COLLECTIONORALIAS, cluster.getZkServer().getZkAddress())
+        .withFunctionName("search", CloudSolrStream.class)
+        .withFunctionName("null", NullStream.class);
+
+    // Basic test
+    stream = factory.constructStream("null(search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\"), by=\"a_i asc\")");
+    tuples = getTuples(stream);
+    assertTrue(tuples.size() == 1);
+    assertTrue(tuples.get(0).getLong("nullCount") == 6);
+  }
+
+
+  @Test
+  public void testParallelNullStream() throws Exception {
+
+    new UpdateRequest()
+        .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0")
+        .add(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0")
+        .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3")
+        .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4")
+        .add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1")
+        .add(id, "5", "a_s", "hello1", "a_i", "1", "a_f", "2")
+        .commit(cluster.getSolrClient(), COLLECTIONORALIAS);
+
+    StreamExpression expression;
+    TupleStream stream;
+    List<Tuple> tuples;
+
+    StreamFactory factory = new StreamFactory()
+        .withCollectionZkHost(COLLECTIONORALIAS, cluster.getZkServer().getZkAddress())
+        .withFunctionName("search", CloudSolrStream.class)
+        .withFunctionName("null", NullStream.class)
+        .withFunctionName("parallel", ParallelStream.class);
+
+    // Basic test
+    stream = factory.constructStream("parallel(" + COLLECTIONORALIAS + ", workers=2, sort=\"nullCount desc\", null(search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\", partitionKeys=id), by=\"a_i asc\"))");
+    tuples = getTuples(stream);
+    assertTrue(tuples.size() == 2);
+    long nullCount = 0;
+    for(Tuple t : tuples) {
+      nullCount += t.getLong("nullCount");
+    }
+
+    assertEquals(nullCount, 6L);
+  }
+
+
   @Test
   public void testNulls() throws Exception {
 


[08/50] lucene-solr:jira/solr-8593: SOLR-9902: StandardDirectoryFactory should use Files API for it's move implementation.

Posted by kr...@apache.org.
SOLR-9902: StandardDirectoryFactory should use Files API for it's move implementation.


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

Branch: refs/heads/jira/solr-8593
Commit: 2781145eb3760489922530fd92d5f1d4c35215a9
Parents: 662be93
Author: markrmiller <ma...@apache.org>
Authored: Thu Dec 29 05:29:51 2016 -0500
Committer: markrmiller <ma...@apache.org>
Committed: Thu Dec 29 05:29:51 2016 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  1 +
 .../solr/core/StandardDirectoryFactory.java     | 20 +++++++++++---------
 2 files changed, 12 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2781145e/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 852a306..06566e0 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -222,6 +222,7 @@ Optimizations
   resulting in less produced garbage and 5-7% better performance.
   (yonik)
 
+* SOLR-9902: StandardDirectoryFactory should use Files API for it's move implementation. (Mark Miller)
 
 Bug Fixes
 ----------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2781145e/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java b/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java
index b24be14..1d8793a 100644
--- a/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java
@@ -129,13 +129,14 @@ public class StandardDirectoryFactory extends CachingDirectoryFactory {
     Directory baseToDir = getBaseDir(toDir);
     
     if (baseFromDir instanceof FSDirectory && baseToDir instanceof FSDirectory) {
-      File dir1 = ((FSDirectory) baseFromDir).getDirectory().toFile();
-      File dir2 = ((FSDirectory) baseToDir).getDirectory().toFile();
-      File indexFileInTmpDir = new File(dir1, fileName);
-      File indexFileInIndex = new File(dir2, fileName);
-      boolean success = indexFileInTmpDir.renameTo(indexFileInIndex);
-      if (success) {
-        return;
+  
+      Path path1 = ((FSDirectory) baseFromDir).getDirectory().toAbsolutePath();
+      Path path2 = ((FSDirectory) baseFromDir).getDirectory().toAbsolutePath();
+      
+      try {
+        Files.move(path1.resolve(fileName), path2.resolve(fileName), StandardCopyOption.ATOMIC_MOVE);
+      } catch (AtomicMoveNotSupportedException e) {
+        Files.move(path1.resolve(fileName), path2.resolve(fileName));
       }
     }
 
@@ -148,8 +149,9 @@ public class StandardDirectoryFactory extends CachingDirectoryFactory {
     if (baseDir instanceof FSDirectory) {
       Path path = ((FSDirectory) baseDir).getDirectory().toAbsolutePath();
       try {
-      Files.move(FileSystems.getDefault().getPath(path.toString(), fileName),
-          FileSystems.getDefault().getPath(path.toString(), toName), StandardCopyOption.ATOMIC_MOVE, StandardCopyOption.REPLACE_EXISTING);
+        Files.move(path.resolve(fileName),
+            path.resolve(toName), StandardCopyOption.ATOMIC_MOVE,
+            StandardCopyOption.REPLACE_EXISTING);
       } catch (AtomicMoveNotSupportedException e) {
         Files.move(FileSystems.getDefault().getPath(path.toString(), fileName),
             FileSystems.getDefault().getPath(path.toString(), toName), StandardCopyOption.REPLACE_EXISTING);


[50/50] lucene-solr:jira/solr-8593: Merge branch 'apache-https-master' into jira/solr-8593

Posted by kr...@apache.org.
Merge branch 'apache-https-master' into jira/solr-8593


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

Branch: refs/heads/jira/solr-8593
Commit: 82350229c9c0b8b69a959aa5320f82cbffb9a4f9
Parents: 918fd97 f330678
Author: Kevin Risden <kr...@apache.org>
Authored: Tue Jan 3 12:47:42 2017 -0600
Committer: Kevin Risden <kr...@apache.org>
Committed: Tue Jan 3 12:47:42 2017 -0600

----------------------------------------------------------------------
 build.xml                                       |   40 +-
 dev-tools/idea/lucene/suggest/suggest.iml       |    1 -
 .../lucene/analysis/common/pom.xml.template     |   36 +-
 .../maven/lucene/analysis/icu/pom.xml.template  |   42 +-
 .../lucene/analysis/kuromoji/pom.xml.template   |   38 +-
 .../lucene/analysis/morfologik/pom.xml.template |   38 +-
 .../lucene/analysis/phonetic/pom.xml.template   |   38 +-
 .../maven/lucene/analysis/pom.xml.template      |   36 +-
 .../lucene/analysis/smartcn/pom.xml.template    |   36 +-
 .../lucene/analysis/stempel/pom.xml.template    |   36 +-
 .../maven/lucene/analysis/uima/pom.xml.template |   36 +-
 .../lucene/backward-codecs/pom.xml.template     |   36 +-
 .../maven/lucene/benchmark/pom.xml.template     |   36 +-
 .../lucene/classification/pom.xml.template      |   36 +-
 dev-tools/maven/lucene/codecs/pom.xml.template  |   36 +-
 .../lucene/codecs/src/java/pom.xml.template     |   36 +-
 .../lucene/codecs/src/test/pom.xml.template     |   36 +-
 dev-tools/maven/lucene/core/pom.xml.template    |   36 +-
 .../maven/lucene/core/src/java/pom.xml.template |   36 +-
 .../maven/lucene/core/src/test/pom.xml.template |   36 +-
 dev-tools/maven/lucene/demo/pom.xml.template    |   36 +-
 .../maven/lucene/expressions/pom.xml.template   |   36 +-
 dev-tools/maven/lucene/facet/pom.xml.template   |   36 +-
 .../maven/lucene/grouping/pom.xml.template      |   36 +-
 .../maven/lucene/highlighter/pom.xml.template   |   36 +-
 dev-tools/maven/lucene/join/pom.xml.template    |   36 +-
 dev-tools/maven/lucene/memory/pom.xml.template  |   36 +-
 dev-tools/maven/lucene/misc/pom.xml.template    |   36 +-
 dev-tools/maven/lucene/pom.xml.template         |   36 +-
 dev-tools/maven/lucene/queries/pom.xml.template |   36 +-
 .../maven/lucene/queryparser/pom.xml.template   |   36 +-
 .../maven/lucene/replicator/pom.xml.template    |   38 +-
 dev-tools/maven/lucene/sandbox/pom.xml.template |   36 +-
 .../lucene/spatial-extras/pom.xml.template      |   36 +-
 dev-tools/maven/lucene/spatial/pom.xml.template |   38 +-
 .../maven/lucene/spatial3d/pom.xml.template     |   36 +-
 dev-tools/maven/lucene/suggest/pom.xml.template |   36 +-
 .../lucene/test-framework/pom.xml.template      |   36 +-
 dev-tools/maven/pom.xml.template                |   36 +-
 .../contrib/analysis-extras/pom.xml.template    |   36 +-
 .../solr/contrib/analytics/pom.xml.template     |   36 +-
 .../solr/contrib/clustering/pom.xml.template    |   36 +-
 .../dataimporthandler-extras/pom.xml.template   |   36 +-
 .../contrib/dataimporthandler/pom.xml.template  |   36 +-
 .../solr/contrib/extraction/pom.xml.template    |   36 +-
 .../maven/solr/contrib/langid/pom.xml.template  |   36 +-
 .../maven/solr/contrib/ltr/pom.xml.template     |   36 +-
 .../solr/contrib/map-reduce/pom.xml.template    |   36 +-
 .../contrib/morphlines-cell/pom.xml.template    |   36 +-
 .../contrib/morphlines-core/pom.xml.template    |   36 +-
 dev-tools/maven/solr/contrib/pom.xml.template   |   36 +-
 .../maven/solr/contrib/uima/pom.xml.template    |   36 +-
 .../solr/contrib/velocity/pom.xml.template      |   36 +-
 dev-tools/maven/solr/core/pom.xml.template      |   36 +-
 .../maven/solr/core/src/java/pom.xml.template   |   36 +-
 .../maven/solr/core/src/test/pom.xml.template   |   36 +-
 dev-tools/maven/solr/pom.xml.template           |   36 +-
 dev-tools/maven/solr/solrj/pom.xml.template     |   36 +-
 .../maven/solr/solrj/src/java/pom.xml.template  |   36 +-
 .../maven/solr/solrj/src/test/pom.xml.template  |   36 +-
 .../maven/solr/test-framework/pom.xml.template  |   36 +-
 lucene/CHANGES.txt                              |   52 +
 .../lucene/analysis/custom/CustomAnalyzer.java  |    2 +-
 .../analysis/synonym/FlattenGraphFilter.java    |  417 ++++
 .../synonym/FlattenGraphFilterFactory.java      |   44 +
 .../lucene/analysis/synonym/SynonymFilter.java  |    4 +
 .../analysis/synonym/SynonymFilterFactory.java  |    4 +
 .../analysis/synonym/SynonymGraphFilter.java    |  586 ++++++
 .../synonym/SynonymGraphFilterFactory.java      |  204 ++
 .../lucene/analysis/synonym/SynonymMap.java     |    7 +-
 .../lucene/analysis/util/CharTokenizer.java     |    6 +-
 ...ache.lucene.analysis.util.TokenFilterFactory |    2 +
 .../lucene/analysis/core/TestRandomChains.java  |  177 +-
 .../analysis/custom/TestCustomAnalyzer.java     |   21 +
 .../apache/lucene/analysis/custom/mapping1.txt  |    1 +
 .../apache/lucene/analysis/custom/mapping2.txt  |    1 +
 .../miscellaneous/TestWordDelimiterFilter.java  |   56 +-
 .../synonym/TestFlattenGraphFilter.java         |  284 +++
 .../synonym/TestSynonymGraphFilter.java         | 1956 ++++++++++++++++++
 .../PackedTokenAttributeImpl.java               |    9 +-
 .../apache/lucene/codecs/DocValuesConsumer.java |   10 +-
 .../org/apache/lucene/codecs/NormsConsumer.java |    2 +-
 .../lucene/codecs/StoredFieldsWriter.java       |    2 +-
 .../apache/lucene/codecs/TermVectorsWriter.java |    2 +-
 .../CompressingStoredFieldsWriter.java          |    2 +-
 .../org/apache/lucene/document/DoublePoint.java |   30 +-
 .../org/apache/lucene/document/FloatPoint.java  |   30 +-
 .../org/apache/lucene/index/DocIDMerger.java    |  211 +-
 .../lucene/index/DocumentsWriterPerThread.java  |   16 +-
 .../lucene/index/MappingMultiPostingsEnum.java  |    2 +-
 .../TrackingTmpOutputDirectoryWrapper.java      |    2 +-
 .../org/apache/lucene/search/DoubleValues.java  |   38 +
 .../lucene/search/DoubleValuesSource.java       |  313 +++
 .../org/apache/lucene/search/GraphQuery.java    |  136 ++
 .../lucene/search/LeafFieldComparator.java      |    2 +-
 .../org/apache/lucene/search/LongValues.java    |   38 +
 .../apache/lucene/search/LongValuesSource.java  |  217 ++
 .../lucene/search/MultiCollectorManager.java    |  105 +
 .../lucene/search/SimpleFieldComparator.java    |    2 +-
 .../org/apache/lucene/store/MMapDirectory.java  |  124 +-
 .../org/apache/lucene/util/QueryBuilder.java    |  113 +-
 .../apache/lucene/util/automaton/Automaton.java |    5 +-
 .../lucene/util/automaton/Operations.java       |    6 +-
 .../apache/lucene/util/automaton/StatePair.java |    4 +-
 .../org/apache/lucene/util/bkd/BKDWriter.java   |   53 +-
 .../graph/GraphTokenStreamFiniteStrings.java    |  230 ++
 .../apache/lucene/util/graph/package-info.java  |   21 +
 .../lucene/index/Test4GBStoredFields.java       |    2 +
 .../apache/lucene/index/TestDocIDMerger.java    |    4 +-
 .../apache/lucene/index/TestIndexWriter.java    |    3 +-
 .../lucene/search/TestDoubleValuesSource.java   |  167 ++
 .../apache/lucene/search/TestGraphQuery.java    |   79 +
 .../apache/lucene/search/TestLRUQueryCache.java |    7 +
 .../lucene/search/TestLongValuesSource.java     |  149 ++
 .../apache/lucene/search/TestPointQueries.java  |   28 +
 .../org/apache/lucene/store/TestDirectory.java  |   13 +-
 .../apache/lucene/store/TestMmapDirectory.java  |    3 +-
 .../org/apache/lucene/store/TestMultiMMap.java  |    3 +-
 .../apache/lucene/util/TestQueryBuilder.java    |   15 +-
 .../org/apache/lucene/util/bkd/TestBKD.java     |   48 +-
 .../org/apache/lucene/util/fst/Test2BFST.java   |    2 +
 .../TestGraphTokenStreamFiniteStrings.java      |  217 ++
 .../org/apache/lucene/facet/DrillSideways.java  |  338 ++-
 .../lucene/facet/FacetsCollectorManager.java    |   55 +
 .../apache/lucene/facet/TestDrillSideways.java  |  315 +--
 .../lucene/facet/TestParallelDrillSideways.java |   90 +
 .../lucene/search/grouping/TestGrouping.java    |    9 -
 lucene/ivy-versions.properties                  |    3 +
 .../queryparser/classic/QueryParserBase.java    |   42 +-
 .../classic/TestMultiFieldQueryParser.java      |   11 +-
 .../queryparser/classic/TestQueryParser.java    |  131 +-
 .../lucene/document/TestHalfFloatPoint.java     |    2 +
 .../lucene/spatial3d/geom/GeoBBoxFactory.java   |   34 +-
 .../lucene/spatial3d/geom/GeoCircleFactory.java |    2 +-
 .../spatial3d/geom/GeoPolygonFactory.java       |    6 +-
 .../apache/lucene/spatial3d/geom/Vector.java    |    4 +
 lucene/suggest/build.xml                        |    6 +-
 .../analyzing/AnalyzingInfixSuggester.java      |  137 +-
 .../suggest/analyzing/FreeTextSuggester.java    |   45 -
 .../analysis/BaseTokenStreamTestCase.java       |   37 +-
 .../apache/lucene/analysis/MockTokenizer.java   |    3 +-
 .../index/BaseStoredFieldsFormatTestCase.java   |    2 +
 .../org/apache/lucene/index/RandomCodec.java    |    2 +-
 .../org/apache/lucene/util/LuceneTestCase.java  |  110 +-
 .../org/apache/lucene/util/RamUsageTester.java  |  102 +-
 lucene/tools/junit4/tests.policy                |    1 -
 solr/CHANGES.txt                                |   59 +-
 solr/bin/solr                                   |   26 +-
 solr/bin/solr.cmd                               |   36 +-
 .../solr/handler/dataimport/JdbcDataSource.java |    4 +-
 .../handler/dataimport/SolrEntityProcessor.java |  192 +-
 .../dataimport/MockSolrEntityProcessor.java     |   18 +-
 .../handler/dataimport/TestJdbcDataSource.java  |   43 +-
 .../TestSolrEntityProcessorEndToEnd.java        |   27 +-
 .../dataimport/TestSolrEntityProcessorUnit.java |   70 +
 solr/contrib/ltr/README.md                      |    2 +-
 .../LTRFeatureLoggerTransformerFactory.java     |   29 +-
 .../solr/collection1/conf/solrconfig-ltr.xml    |    1 +
 .../apache/solr/ltr/TestLTRQParserExplain.java  |    2 +-
 .../apache/solr/ltr/TestLTRQParserPlugin.java   |    5 +-
 .../org/apache/solr/ltr/TestLTRWithFacet.java   |    2 +-
 .../org/apache/solr/ltr/TestLTRWithSort.java    |    2 +-
 .../org/apache/solr/ltr/TestRerankBase.java     |   54 +-
 .../solr/ltr/TestSelectiveWeightCreation.java   |    2 +-
 .../ltr/feature/TestEdisMaxSolrFeature.java     |    2 +-
 .../solr/ltr/feature/TestExternalFeatures.java  |   37 +-
 .../ltr/feature/TestExternalValueFeatures.java  |   21 +-
 .../solr/ltr/feature/TestFeatureLogging.java    |   30 +-
 .../ltr/feature/TestFeatureLtrScoringModel.java |    2 +-
 .../solr/ltr/feature/TestFeatureStore.java      |    2 +-
 .../ltr/feature/TestFieldLengthFeature.java     |    2 +-
 .../solr/ltr/feature/TestFieldValueFeature.java |    2 +-
 .../solr/ltr/feature/TestFilterSolrFeature.java |    6 +-
 .../ltr/feature/TestNoMatchSolrFeature.java     |   94 +-
 .../ltr/feature/TestOriginalScoreFeature.java   |    2 +-
 .../solr/ltr/feature/TestRankingFeature.java    |    2 +-
 .../ltr/feature/TestUserTermScoreWithQ.java     |    2 +-
 .../ltr/feature/TestUserTermScorerQuery.java    |    2 +-
 .../ltr/feature/TestUserTermScorereQDF.java     |    2 +-
 .../solr/ltr/feature/TestValueFeature.java      |    2 +-
 .../apache/solr/ltr/model/TestLinearModel.java  |    2 +-
 .../model/TestMultipleAdditiveTreesModel.java   |    2 +-
 .../solr/ltr/store/rest/TestModelManager.java   |    2 +-
 .../store/rest/TestModelManagerPersistence.java |    2 +-
 .../java/org/apache/solr/cloud/Overseer.java    |    2 +-
 .../org/apache/solr/core/CoreContainer.java     |   23 +-
 .../org/apache/solr/core/DirectoryFactory.java  |   26 +
 .../apache/solr/core/HdfsDirectoryFactory.java  |   29 +
 .../solr/core/MetricsDirectoryFactory.java      |  511 +++++
 .../src/java/org/apache/solr/core/SolrCore.java |   24 +-
 .../apache/solr/core/SolrDeletionPolicy.java    |    6 +
 .../org/apache/solr/core/SolrInfoMBean.java     |    2 +-
 .../solr/core/StandardDirectoryFactory.java     |   45 +-
 .../org/apache/solr/handler/IndexFetcher.java   |   25 +-
 .../org/apache/solr/handler/StreamHandler.java  |    3 +-
 .../solr/handler/admin/CoreAdminHandler.java    |   13 +-
 .../solr/handler/admin/LukeRequestHandler.java  |   18 +-
 .../admin/SegmentsInfoRequestHandler.java       |    1 +
 .../handler/component/HttpShardHandler.java     |   48 +-
 .../component/HttpShardHandlerFactory.java      |  154 +-
 .../solr/handler/component/QueryComponent.java  |   18 +-
 .../component/QueryElevationComponent.java      |    4 +-
 .../solr/handler/component/ResponseBuilder.java |    1 -
 .../solr/handler/component/SearchHandler.java   |    2 +-
 .../solr/handler/component/ShardHandler.java    |    5 +-
 .../solr/highlight/UnifiedSolrHighlighter.java  |  459 ++--
 .../solr/metrics/OperatingSystemMetricSet.java  |   92 +
 .../metrics/reporters/SolrGangliaReporter.java  |  144 ++
 .../metrics/reporters/SolrGraphiteReporter.java |  129 ++
 .../metrics/reporters/SolrSlf4jReporter.java    |  127 ++
 .../apache/solr/parser/SolrQueryParserBase.java |   16 +-
 .../solr/response/JSONResponseWriter.java       |   86 +-
 .../org/apache/solr/schema/SchemaManager.java   |   99 +-
 .../solr/search/CollapsingQParserPlugin.java    |   14 +-
 .../search/grouping/GroupingSpecification.java  |   18 +-
 .../apache/solr/servlet/SolrDispatchFilter.java |    8 +-
 .../solr/spelling/DirectSolrSpellChecker.java   |   18 +-
 .../org/apache/solr/update/SolrIndexConfig.java |   16 +-
 .../org/apache/solr/update/SolrIndexWriter.java |  144 ++
 .../apache/solr/update/UpdateShardHandler.java  |   72 +-
 .../src/java/org/apache/solr/util/SolrCLI.java  |   65 +-
 .../src/java/org/apache/solr/util/TimeOut.java  |   13 +-
 .../stats/InstrumentedHttpRequestExecutor.java  |  129 ++
 ...entedPoolingHttpClientConnectionManager.java |  113 +
 .../org/apache/solr/util/stats/MetricUtils.java |   88 +-
 .../schema-minimal-with-another-uniqkey.xml     |   23 +
 .../conf/solrconfig-indexmetrics.xml            |   57 +
 .../test-files/solr/solr-gangliareporter.xml    |   32 +
 .../test-files/solr/solr-graphitereporter.xml   |   31 +
 .../src/test-files/solr/solr-slf4jreporter.xml  |   35 +
 .../TestReversedWildcardFilterFactory.java      |   56 +-
 .../apache/solr/cloud/CdcrBootstrapTest.java    |   25 +-
 .../solr/cloud/CollectionsAPISolrJTest.java     |  470 ++---
 .../DeleteLastCustomShardedReplicaTest.java     |  104 +-
 .../org/apache/solr/cloud/DeleteShardTest.java  |  211 +-
 .../solr/cloud/DocValuesNotIndexedTest.java     |   10 -
 .../cloud/LeaderFailoverAfterPartitionTest.java |    2 +-
 .../cloud/LeaderFailureAfterFreshStartTest.java |   68 +-
 .../cloud/OverseerModifyCollectionTest.java     |   92 +-
 .../apache/solr/cloud/OverseerRolesTest.java    |  165 +-
 .../apache/solr/cloud/OverseerStatusTest.java   |   55 +-
 .../solr/cloud/PeerSyncReplicationTest.java     |   55 +-
 .../apache/solr/cloud/RemoteQueryErrorTest.java |   53 +-
 .../solr/cloud/TestDownShardTolerantSearch.java |   40 +-
 .../TestExclusionRuleCollectionAccess.java      |   38 +-
 .../apache/solr/cloud/TestStressLiveNodes.java  |    2 +-
 .../solr/core/MockShardHandlerFactory.java      |    2 +-
 .../test/org/apache/solr/core/TestConfig.java   |    2 +
 .../solr/handler/TestReplicationHandler.java    |   20 +-
 .../admin/CoreMergeIndexesAdminHandlerTest.java |   10 +-
 .../solr/handler/admin/MetricsHandlerTest.java  |    6 +-
 .../admin/SegmentsInfoRequestHandlerTest.java   |   17 +-
 .../solr/handler/admin/StatsReloadRaceTest.java |   82 +-
 .../highlight/TestUnifiedSolrHighlighter.java   |   10 +
 .../org/apache/solr/metrics/JvmMetricsTest.java |   68 +
 .../reporters/SolrGangliaReporterTest.java      |   82 +
 .../reporters/SolrGraphiteReporterTest.java     |  116 ++
 .../reporters/SolrSlf4jReporterTest.java        |   77 +
 .../apache/solr/response/JSONWriterTest.java    |   20 +-
 .../TestSubQueryTransformerDistrib.java         |   55 +-
 .../PKIAuthenticationIntegrationTest.java       |   40 +-
 .../TestSolrCloudWithHadoopAuthPlugin.java      |    4 +-
 .../org/apache/solr/update/AutoCommitTest.java  |    2 +-
 .../apache/solr/update/SoftAutoCommitTest.java  |    2 +-
 .../apache/solr/update/SolrIndexConfigTest.java |    1 +
 .../solr/update/SolrIndexMetricsTest.java       |   94 +
 .../apache/solr/util/stats/MetricUtilsTest.java |   24 +-
 solr/licenses/gmetric4j-1.0.7.jar.sha1          |    1 +
 solr/licenses/gmetric4j-LICENSE-BSD.txt         |   31 +
 solr/licenses/gmetric4j-NOTICE.txt              |    0
 solr/licenses/metrics-ganglia-3.1.2.jar.sha1    |    1 +
 solr/licenses/metrics-ganglia-LICENSE-ASL.txt   |  203 ++
 solr/licenses/metrics-ganglia-NOTICE.txt        |   12 +
 solr/licenses/metrics-graphite-3.1.2.jar.sha1   |    1 +
 solr/licenses/metrics-graphite-LICENSE-ASL.txt  |  203 ++
 solr/licenses/metrics-graphite-NOTICE.txt       |   12 +
 solr/server/ivy.xml                             |    3 +
 .../solr/client/solrj/impl/HttpClientUtil.java  |   35 +-
 .../client/solrj/io/stream/CloudSolrStream.java |    4 +
 .../solr/client/solrj/io/stream/NullStream.java |  155 ++
 .../client/solrj/io/stream/PriorityStream.java  |  161 ++
 .../solrj/request/CollectionAdminRequest.java   |    2 +
 .../solr/common/params/HighlightParams.java     |    2 +-
 .../client/solrj/impl/CloudSolrClientTest.java  |    2 +-
 .../solrj/io/stream/StreamExpressionTest.java   |  224 ++
 .../solr/common/params/CommonParamsTest.java    |    2 +
 .../solr/cloud/AbstractDistribZkTestBase.java   |   31 +-
 .../java/org/apache/solr/cloud/ChaosMonkey.java |    2 -
 .../component/TrackingShardHandlerFactory.java  |    2 +-
 289 files changed, 13357 insertions(+), 3649 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/82350229/lucene/ivy-versions.properties
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/82350229/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/82350229/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
----------------------------------------------------------------------


[30/50] lucene-solr:jira/solr-8593: SOLR-9668: introduce cursorMark='true' for SolrEntityProcessor

Posted by kr...@apache.org.
SOLR-9668: introduce cursorMark='true' for SolrEntityProcessor


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

Branch: refs/heads/jira/solr-8593
Commit: cc862d8e67f32d5447599d265f5d126541ed92c9
Parents: 26ee8e9
Author: Mikhail Khludnev <mk...@apache.org>
Authored: Tue Dec 27 15:34:12 2016 +0300
Committer: Mikhail Khludnev <mk...@apache.org>
Committed: Fri Dec 30 20:50:53 2016 +0300

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   2 +
 .../handler/dataimport/SolrEntityProcessor.java | 192 ++++++++++++-------
 .../dataimport/MockSolrEntityProcessor.java     |  18 +-
 .../TestSolrEntityProcessorEndToEnd.java        |  27 ++-
 .../dataimport/TestSolrEntityProcessorUnit.java |  70 +++++++
 5 files changed, 234 insertions(+), 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cc862d8e/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 59dde90..874ac81 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -206,6 +206,8 @@ New Features
 
 * SOLR-9891: Add mkroot command to bin/solr and bin/solr.cmd (Erick Erickson)
 
+* SOLR-9668,SOLR-7197: introduce cursorMark='true' in SolrEntityProcessor (Yegor Kozlov, Raveendra Yerraguntl via Mikhail Khludnev)
+
 Optimizations
 ----------------------
 * SOLR-9704: Facet Module / JSON Facet API: Optimize blockChildren facets that have

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cc862d8e/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrEntityProcessor.java
----------------------------------------------------------------------
diff --git a/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrEntityProcessor.java b/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrEntityProcessor.java
index 5e62731..6d8726f 100644
--- a/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrEntityProcessor.java
+++ b/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrEntityProcessor.java
@@ -16,6 +16,18 @@
  */
 package org.apache.solr.handler.dataimport;
 
+import static org.apache.solr.handler.dataimport.DataImportHandlerException.SEVERE;
+import static org.apache.solr.handler.dataimport.DataImportHandlerException.wrapAndThrow;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
 import org.apache.http.client.HttpClient;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrQuery;
@@ -27,22 +39,12 @@ import org.apache.solr.client.solrj.impl.XMLResponseParser;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.CursorMarkParams;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-
-import static org.apache.solr.handler.dataimport.DataImportHandlerException.SEVERE;
-import static org.apache.solr.handler.dataimport.DataImportHandlerException.wrapAndThrow;
-
 /**
  * <p>
  * An implementation of {@link EntityProcessor} which fetches values from a
@@ -139,81 +141,53 @@ public class SolrEntityProcessor extends EntityProcessorBase {
    * The following method changes the rowIterator mutable field. It requires
    * external synchronization. 
    */
-  private void buildIterator() {
+  protected void buildIterator() {
     if (rowIterator != null)  {
       SolrDocumentListIterator documentListIterator = (SolrDocumentListIterator) rowIterator;
       if (!documentListIterator.hasNext() && documentListIterator.hasMoreRows()) {
-        SolrDocumentList solrDocumentList = doQuery(documentListIterator
-            .getStart() + documentListIterator.getSize());
-        if (solrDocumentList != null) {
-          rowIterator = new SolrDocumentListIterator(solrDocumentList);
-        }
+        nextPage();
       }
-    } else  {
-      SolrDocumentList solrDocumentList = doQuery(0);
-      if (solrDocumentList != null) {
-        rowIterator = new SolrDocumentListIterator(solrDocumentList);
-      }
-      return;
+    } else {
+      Boolean cursor = new Boolean(context
+          .getResolvedEntityAttribute(CursorMarkParams.CURSOR_MARK_PARAM));
+      rowIterator = !cursor ? new SolrDocumentListIterator(new SolrDocumentList())
+          : new SolrDocumentListCursor(new SolrDocumentList(), CursorMarkParams.CURSOR_MARK_START);
+      nextPage();
     }
   }
   
-  protected SolrDocumentList doQuery(int start) {
-    this.queryString = context.getResolvedEntityAttribute(QUERY);
-    if (this.queryString == null) {
-      throw new DataImportHandlerException(
-          DataImportHandlerException.SEVERE,
-          "SolrEntityProcessor: parameter 'query' is required"
-      );
-    }
-
-    String rowsP = context.getResolvedEntityAttribute(CommonParams.ROWS);
-    if (rowsP != null) {
-      rows = Integer.parseInt(rowsP);
-    }
+  protected void nextPage() {
+    ((SolrDocumentListIterator)rowIterator).doQuery();
+  }
 
-    String fqAsString = context.getResolvedEntityAttribute(CommonParams.FQ);
-    if (fqAsString != null) {
-      this.filterQueries = fqAsString.split(",");
-    }
+  class SolrDocumentListCursor extends SolrDocumentListIterator {
+    
+    private final String cursorMark;
 
-    String fieldsAsString = context.getResolvedEntityAttribute(CommonParams.FL);
-    if (fieldsAsString != null) {
-      this.fields = fieldsAsString.split(",");
-    }
-    this.requestHandler = context.getResolvedEntityAttribute(CommonParams.QT);
-    String timeoutAsString = context.getResolvedEntityAttribute(TIMEOUT);
-    if (timeoutAsString != null) {
-      this.timeout = Integer.parseInt(timeoutAsString);
+    public SolrDocumentListCursor(SolrDocumentList solrDocumentList, String cursorMark) {
+      super(solrDocumentList);
+      this.cursorMark = cursorMark;
     }
 
-    SolrQuery solrQuery = new SolrQuery(queryString);
-    solrQuery.setRows(rows);
-    solrQuery.setStart(start);
-    if (fields != null) {
-      for (String field : fields) {
-        solrQuery.addField(field);
+    @Override
+    protected void passNextPage(SolrQuery solrQuery) {
+      String timeoutAsString = context.getResolvedEntityAttribute(TIMEOUT);
+      if (timeoutAsString != null) {
+        throw new DataImportHandlerException(SEVERE,"cursorMark can't be used with timeout");
       }
+      
+      solrQuery.set(CursorMarkParams.CURSOR_MARK_PARAM, cursorMark);
     }
-    solrQuery.setRequestHandler(requestHandler);
-    solrQuery.setFilterQueries(filterQueries);
-    solrQuery.setTimeAllowed(timeout * 1000);
     
-    QueryResponse response = null;
-    try {
-      response = solrClient.query(solrQuery);
-    } catch (SolrServerException | IOException e) {
-      if (ABORT.equals(onError)) {
-        wrapAndThrow(SEVERE, e);
-      } else if (SKIP.equals(onError)) {
-        wrapAndThrow(DataImportHandlerException.SKIP_ROW, e);
-      }
+    @Override
+    protected Iterator<Map<String,Object>> createNextPageIterator(QueryResponse response) {
+      return
+          new SolrDocumentListCursor(response.getResults(),
+              response.getNextCursorMark()) ;
     }
-    
-    return response == null ? null : response.getResults();
   }
   
-  private static class SolrDocumentListIterator implements Iterator<Map<String,Object>> {
+  class SolrDocumentListIterator implements Iterator<Map<String,Object>> {
     
     private final int start;
     private final int size;
@@ -230,6 +204,84 @@ public class SolrEntityProcessor extends EntityProcessorBase {
       this.size = solrDocumentList.size();
     }
 
+    protected QueryResponse doQuery() {
+      SolrEntityProcessor.this.queryString = context.getResolvedEntityAttribute(QUERY);
+      if (SolrEntityProcessor.this.queryString == null) {
+        throw new DataImportHandlerException(
+            DataImportHandlerException.SEVERE,
+            "SolrEntityProcessor: parameter 'query' is required"
+        );
+      }
+
+      String rowsP = context.getResolvedEntityAttribute(CommonParams.ROWS);
+      if (rowsP != null) {
+        rows = Integer.parseInt(rowsP);
+      }
+
+      String sortParam = context.getResolvedEntityAttribute(CommonParams.SORT);
+      
+      String fqAsString = context.getResolvedEntityAttribute(CommonParams.FQ);
+      if (fqAsString != null) {
+        SolrEntityProcessor.this.filterQueries = fqAsString.split(",");
+      }
+
+      String fieldsAsString = context.getResolvedEntityAttribute(CommonParams.FL);
+      if (fieldsAsString != null) {
+        SolrEntityProcessor.this.fields = fieldsAsString.split(",");
+      }
+      SolrEntityProcessor.this.requestHandler = context.getResolvedEntityAttribute(CommonParams.QT);
+     
+
+      SolrQuery solrQuery = new SolrQuery(queryString);
+      solrQuery.setRows(rows);
+      
+      if (sortParam!=null) {
+        solrQuery.setParam(CommonParams.SORT, sortParam);
+      }
+      
+      passNextPage(solrQuery);
+      
+      if (fields != null) {
+        for (String field : fields) {
+          solrQuery.addField(field);
+        }
+      }
+      solrQuery.setRequestHandler(requestHandler);
+      solrQuery.setFilterQueries(filterQueries);
+      
+      
+      QueryResponse response = null;
+      try {
+        response = solrClient.query(solrQuery);
+      } catch (SolrServerException | IOException | SolrException e) {
+        if (ABORT.equals(onError)) {
+          wrapAndThrow(SEVERE, e);
+        } else if (SKIP.equals(onError)) {
+          wrapAndThrow(DataImportHandlerException.SKIP_ROW, e);
+        }
+      }
+      
+      if (response != null) {
+        SolrEntityProcessor.this.rowIterator = createNextPageIterator(response);
+      }
+      return response;
+    }
+
+    protected Iterator<Map<String,Object>> createNextPageIterator(QueryResponse response) {
+      return new SolrDocumentListIterator(response.getResults());
+    }
+
+    protected void passNextPage(SolrQuery solrQuery) {
+      String timeoutAsString = context.getResolvedEntityAttribute(TIMEOUT);
+      if (timeoutAsString != null) {
+        SolrEntityProcessor.this.timeout = Integer.parseInt(timeoutAsString);
+      }
+      
+      solrQuery.setTimeAllowed(timeout * 1000);
+      
+      solrQuery.setStart(getStart() + getSize());
+    }
+    
     @Override
     public boolean hasNext() {
       return solrDocumentIterator.hasNext();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cc862d8e/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/MockSolrEntityProcessor.java
----------------------------------------------------------------------
diff --git a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/MockSolrEntityProcessor.java b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/MockSolrEntityProcessor.java
index 4ebca30..42e5f7d 100644
--- a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/MockSolrEntityProcessor.java
+++ b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/MockSolrEntityProcessor.java
@@ -29,16 +29,28 @@ public class MockSolrEntityProcessor extends SolrEntityProcessor {
   private int queryCount = 0;
 
   private int rows;
+  
+  private int start = 0;
 
   public MockSolrEntityProcessor(List<SolrTestCaseJ4.Doc> docsData, int rows) {
     this.docsData = docsData;
     this.rows = rows;
   }
 
+  //@Override
+  //protected SolrDocumentList doQuery(int start) {
+  //  queryCount++;
+  //  return getDocs(start, rows);
+ // }
+  
   @Override
-  protected SolrDocumentList doQuery(int start) {
-    queryCount++;
-    return getDocs(start, rows);
+  protected void buildIterator() {
+    if (rowIterator==null || (!rowIterator.hasNext() && ((SolrDocumentListIterator)rowIterator).hasMoreRows())){
+      queryCount++;
+      SolrDocumentList docs = getDocs(start, rows);
+      rowIterator = new SolrDocumentListIterator(docs);
+      start += docs.size();
+    }
   }
 
   private SolrDocumentList getDocs(int start, int rows) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cc862d8e/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java
----------------------------------------------------------------------
diff --git a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java
index 8ef94c0..9e104ee 100644
--- a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java
+++ b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java
@@ -34,6 +34,8 @@ import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.nio.file.Files;
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -179,7 +181,7 @@ public class TestSolrEntityProcessorEndToEnd extends AbstractDataImportHandlerTe
     
     try {
       addDocumentsToSolr(generateSolrDocuments(7));
-      runFullImport(generateDIHConfig("query='*:*' fl='id' rows='2'", false));
+      runFullImport(generateDIHConfig("query='*:*' fl='id' rows='2'"+(random().nextBoolean() ?" cursorMark='true' sort='id asc'":""), false));
     } catch (Exception e) {
       LOG.error(e.getMessage(), e);
       fail(e.getMessage());
@@ -252,7 +254,8 @@ public class TestSolrEntityProcessorEndToEnd extends AbstractDataImportHandlerTe
     assertQ(req("*:*"), "//result[@numFound='0']");
     
     try {
-      runFullImport(generateDIHConfig("query='bogus:3' rows='2' fl='id,desc' onError='abort'", false));
+      runFullImport(generateDIHConfig("query='bogus:3' rows='2' fl='id,desc' onError='"+
+            (random().nextBoolean() ? "abort" : "justtogetcoverage")+"'", false));
     } catch (Exception e) {
       LOG.error(e.getMessage(), e);
       fail(e.getMessage());
@@ -260,7 +263,27 @@ public class TestSolrEntityProcessorEndToEnd extends AbstractDataImportHandlerTe
     
     assertQ(req("*:*"), "//result[@numFound='0']");
   }
+  
+  public void testCursorMarkNoSort() throws SolrServerException, IOException {
+    assertQ(req("*:*"), "//result[@numFound='0']");
+    addDocumentsToSolr(generateSolrDocuments(7));
+    try {     
+      List<String> errors = Arrays.asList("sort='id'", //wrong sort spec
+          "", //no sort spec
+          "sort='id asc' timeout='12345'"); // sort is fine, but set timeout
+      Collections.shuffle(errors, random());
+      String attrs = "query='*:*' rows='2' fl='id,desc' cursorMark='true' "
+                                                            + errors.get(0);
+      runFullImport(generateDIHConfig(attrs,
+            false));
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      fail(e.getMessage());
+    }
     
+    assertQ(req("*:*"), "//result[@numFound='0']");
+  }
+  
   private static List<Map<String,Object>> generateSolrDocuments(int num) {
     List<Map<String,Object>> docList = new ArrayList<>();
     for (int i = 1; i <= num; i++) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cc862d8e/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorUnit.java
----------------------------------------------------------------------
diff --git a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorUnit.java b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorUnit.java
index a8fcbb1..a2a9fff 100644
--- a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorUnit.java
+++ b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorUnit.java
@@ -18,11 +18,23 @@ package org.apache.solr.handler.dataimport;
 
 import java.util.*;
 
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.CursorMarkParams;
+import org.apache.solr.handler.dataimport.SolrEntityProcessor.SolrDocumentListIterator;
+import org.junit.Test;
+
 /**
  * Unit test of SolrEntityProcessor. A very basic test outside of the DIH.
  */
 public class TestSolrEntityProcessorUnit extends AbstractDataImportHandlerTestCase {
 
+  private static final class NoNextMockProcessor extends SolrEntityProcessor {
+    @Override
+    protected void nextPage() {
+    }
+  }
+
   private static final String ID = "id";
 
   public void testQuery() {
@@ -85,6 +97,64 @@ public class TestSolrEntityProcessorUnit extends AbstractDataImportHandlerTestCa
       processor.destroy();
     }
   }
+  @Test (expected = DataImportHandlerException.class)
+  public void testNoQuery() {
+    SolrEntityProcessor processor = new SolrEntityProcessor();
+    
+    HashMap<String,String> entityAttrs = new HashMap<String,String>(){{put(SolrEntityProcessor.SOLR_SERVER,"http://route:66/no");}};
+    processor.init(getContext(null, null, null, null, Collections.emptyList(), 
+        entityAttrs));
+    try {
+    processor.buildIterator();
+    }finally {
+      processor.destroy();
+    }
+  }
+  
+  public void testPagingQuery() {
+    SolrEntityProcessor processor = new NoNextMockProcessor() ;
+    
+    HashMap<String,String> entityAttrs = new HashMap<String,String>(){{
+      put(SolrEntityProcessor.SOLR_SERVER,"http://route:66/no");
+      if (random().nextBoolean()) {
+        List<String> noCursor = Arrays.asList("","false",CursorMarkParams.CURSOR_MARK_START);//only 'true' not '*'
+        Collections.shuffle(noCursor, random());
+        put(CursorMarkParams.CURSOR_MARK_PARAM,  noCursor.get(0));
+      }}};
+    processor.init(getContext(null, null, null, null, Collections.emptyList(), 
+        entityAttrs));
+    try {
+    processor.buildIterator();
+    SolrQuery query = new SolrQuery();
+    ((SolrDocumentListIterator) processor.rowIterator).passNextPage(query);
+    assertEquals("0", query.get(CommonParams.START));
+    assertNull( query.get(CursorMarkParams.CURSOR_MARK_PARAM));
+    assertNotNull( query.get(CommonParams.TIME_ALLOWED));
+    }finally {
+      processor.destroy();
+    }
+  }
+  
+  public void testCursorQuery() {
+    SolrEntityProcessor processor = new NoNextMockProcessor() ;
+    
+    HashMap<String,String> entityAttrs = new HashMap<String,String>(){{
+      put(SolrEntityProcessor.SOLR_SERVER,"http://route:66/no");
+      put(CursorMarkParams.CURSOR_MARK_PARAM,"true");
+      }};
+    processor.init(getContext(null, null, null, null, Collections.emptyList(), 
+        entityAttrs));
+    try {
+    processor.buildIterator();
+    SolrQuery query = new SolrQuery();
+    ((SolrDocumentListIterator) processor.rowIterator).passNextPage(query);
+    assertNull(query.get(CommonParams.START));
+    assertEquals(CursorMarkParams.CURSOR_MARK_START, query.get(CursorMarkParams.CURSOR_MARK_PARAM));
+    assertNull( query.get(CommonParams.TIME_ALLOWED));
+    }finally {
+      processor.destroy();
+    }
+  }
 
   private List<Doc> generateUniqueDocs(int numDocs) {
     List<FldType> types = new ArrayList<>();


[26/50] lucene-solr:jira/solr-8593: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/lucene-solr

Posted by kr...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/lucene-solr


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

Branch: refs/heads/jira/solr-8593
Commit: b584f9c56c3ef22cb54c11fd5051b54f8ffed6c7
Parents: d65c02e 5d042d3
Author: Uwe Schindler <us...@apache.org>
Authored: Thu Dec 29 22:57:39 2016 +0100
Committer: Uwe Schindler <us...@apache.org>
Committed: Thu Dec 29 22:57:39 2016 +0100

----------------------------------------------------------------------
 .../lucene/search/TestDoubleValuesSource.java   |   9 ++
 .../lucene/search/TestLongValuesSource.java     |   9 ++
 .../analyzing/AnalyzingInfixSuggester.java      | 137 +++++++++-------
 solr/CHANGES.txt                                |   4 +
 .../org/apache/solr/handler/StreamHandler.java  |   2 +-
 .../apache/solr/parser/SolrQueryParserBase.java |  14 ++
 .../TestReversedWildcardFilterFactory.java      |  12 ++
 .../solr/client/solrj/io/stream/NullStream.java | 155 +++++++++++++++++++
 .../solrj/io/stream/StreamExpressionTest.java   |  65 ++++++++
 9 files changed, 351 insertions(+), 56 deletions(-)
----------------------------------------------------------------------



[48/50] lucene-solr:jira/solr-8593: LUCENE-6664: be more robust to broken token stream offsets

Posted by kr...@apache.org.
LUCENE-6664: be more robust to broken token stream offsets


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

Branch: refs/heads/jira/solr-8593
Commit: f6fb6941bb62f8d47d653b2ed187ffa0107cd5c5
Parents: b4a002f
Author: Mike McCandless <mi...@apache.org>
Authored: Tue Jan 3 06:47:47 2017 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Tue Jan 3 06:47:47 2017 -0500

----------------------------------------------------------------------
 .../analysis/synonym/FlattenGraphFilter.java    | 31 ++++++++------------
 1 file changed, 12 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f6fb6941/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/FlattenGraphFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/FlattenGraphFilter.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/FlattenGraphFilter.java
index 7ede190..c1fa1f7 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/FlattenGraphFilter.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/FlattenGraphFilter.java
@@ -17,22 +17,6 @@
 
 package org.apache.lucene.analysis.synonym;
 
-/**
- * This filter "casts" token graphs down into a "flat" form,
- * for indexing.   This is an inherently lossy process: nodes (positions)
- * along side paths are forcefully merged.
- *
- * <p>In general this means the output graph will accept token sequences
- * that the input graph did not accept, and will also fail to accept
- * token sequences that the input graph did accept.
- *
- * <p>This is only necessary at indexing time because Lucene cannot yet index
- * an arbitrary token graph.  At search time there are better options, e.g.
- * the experimental <code>TermAutomatonQuery</code> in sandbox.
- *
- * @lucene.experimental
- */
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
@@ -49,7 +33,12 @@ import org.apache.lucene.util.RollingBuffer;
  * Converts an incoming graph token stream, such as one from
  * {@link SynonymGraphFilter}, into a flat form so that
  * all nodes form a single linear chain with no side paths.  Every
- * path through the graph touches every node.
+ * path through the graph touches every node.  This is necessary
+ * when indexing a graph token stream, because the index does not
+ * save {@link PositionLengthAttribute} and so it cannot
+ * preserve the graph structure.  However, at search time,
+ * query parsers can correctly handle the graph and this token
+ * filter should <b>not</b> be used.
  *
  * <p>If the graph was not already flat to start, this
  * is likely a lossy process, i.e. it will often cause the 
@@ -234,7 +223,11 @@ public final class FlattenGraphFilter extends TokenFilter {
         // which would otherwise happen if the replacement has more tokens
         // than the input:
         int startOffset = Math.max(lastStartOffset, output.startOffset);
-        offsetAtt.setOffset(startOffset, outputEndNode.endOffset);
+
+        // We must do this in case the incoming tokens have broken offsets:
+        int endOffset = Math.max(startOffset, outputEndNode.endOffset);
+        
+        offsetAtt.setOffset(startOffset, endOffset);
         lastStartOffset = startOffset;
 
         if (inputNode.nextOut == inputNode.tokens.size()) {
@@ -382,7 +375,7 @@ public final class FlattenGraphFilter extends TokenFilter {
       // NOTE, shady: don't call super.end, because we did already from incrementToken
     }
 
-   clearAttributes();
+    clearAttributes();
     if (done) {
       // On exc, done is false, and we will not have set these:
       posIncAtt.setPositionIncrement(finalPosInc);


[36/50] lucene-solr:jira/solr-8593: SOLR-9154: Fix DirectSolrSpellChecker to work when added through the Config API

Posted by kr...@apache.org.
SOLR-9154: Fix DirectSolrSpellChecker to work when added through the Config API


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

Branch: refs/heads/jira/solr-8593
Commit: 93562da610bf8756351be7720c69872bc1cea727
Parents: fb2800b
Author: anshum <an...@apache.org>
Authored: Sun Jan 1 15:31:02 2017 -0800
Committer: anshum <an...@apache.org>
Committed: Sun Jan 1 15:32:38 2017 -0800

----------------------------------------------------------------------
 solr/CHANGES.txt                                  |  4 +++-
 .../solr/spelling/DirectSolrSpellChecker.java     | 18 +++++++++++-------
 2 files changed, 14 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/93562da6/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 02167f3..7133638 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -299,7 +299,7 @@ Bug Fixes
 
 * SOLR-9699,SOLR-4668: fix exception from core status in parallel with core reload (Mikhail Khludnev)
 
-* SOLR-9859: replication.properties cannot be updated after being written and neither replication.properties or 
+* SOLR-9859: replication.properties cannot be updated after being written and neither replication.properties or
   index.properties are durable in the face of a crash. (Pushkar Raste, Chris de Kok, Cao Manh Dat, Mark Miller)
 
 * SOLR-9901: Implement move in HdfsDirectoryFactory. (Mark Miller)
@@ -308,6 +308,8 @@ Bug Fixes
 
 * SOLR-9495: AIOBE with confusing message for incomplete sort spec in Streaming Expression (Gus Heck, Joel Bernstein)
 
+* SOLR-9154: Fix DirectSolrSpellChecker to work when added through the Config API. (Anshum Gupta)
+
 Other Changes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/93562da6/solr/core/src/java/org/apache/solr/spelling/DirectSolrSpellChecker.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/spelling/DirectSolrSpellChecker.java b/solr/core/src/java/org/apache/solr/spelling/DirectSolrSpellChecker.java
index bbde74a..15fee72 100644
--- a/solr/core/src/java/org/apache/solr/spelling/DirectSolrSpellChecker.java
+++ b/solr/core/src/java/org/apache/solr/spelling/DirectSolrSpellChecker.java
@@ -29,6 +29,7 @@ import org.apache.lucene.search.spell.StringDistance;
 import org.apache.lucene.search.spell.SuggestWord;
 import org.apache.lucene.search.spell.SuggestWordFrequencyComparator;
 import org.apache.lucene.search.spell.SuggestWordQueue;
+import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.search.SolrIndexSearcher;
@@ -93,6 +94,9 @@ public class DirectSolrSpellChecker extends SolrSpellChecker {
   
   @Override
   public String init(NamedList config, SolrCore core) {
+
+    SolrParams params = SolrParams.toSolrParams(config);
+
     LOG.info("init: " + config);
     String name = super.init(config, core);
     
@@ -113,37 +117,37 @@ public class DirectSolrSpellChecker extends SolrSpellChecker {
       sd = core.getResourceLoader().newInstance(distClass, StringDistance.class);
 
     float minAccuracy = DEFAULT_ACCURACY;
-    Float accuracy = (Float) config.get(ACCURACY);
+    Float accuracy = params.getFloat(ACCURACY);
     if (accuracy != null)
       minAccuracy = accuracy;
     
     int maxEdits = DEFAULT_MAXEDITS;
-    Integer edits = (Integer) config.get(MAXEDITS);
+    Integer edits = params.getInt(MAXEDITS);
     if (edits != null)
       maxEdits = edits;
     
     int minPrefix = DEFAULT_MINPREFIX;
-    Integer prefix = (Integer) config.get(MINPREFIX);
+    Integer prefix = params.getInt(MINPREFIX);
     if (prefix != null)
       minPrefix = prefix;
     
     int maxInspections = DEFAULT_MAXINSPECTIONS;
-    Integer inspections = (Integer) config.get(MAXINSPECTIONS);
+    Integer inspections = params.getInt(MAXINSPECTIONS);
     if (inspections != null)
       maxInspections = inspections;
     
     float minThreshold = DEFAULT_THRESHOLD_TOKEN_FREQUENCY;
-    Float threshold = (Float) config.get(THRESHOLD_TOKEN_FREQUENCY);
+    Float threshold = params.getFloat(THRESHOLD_TOKEN_FREQUENCY);
     if (threshold != null)
       minThreshold = threshold;
     
     int minQueryLength = DEFAULT_MINQUERYLENGTH;
-    Integer queryLength = (Integer) config.get(MINQUERYLENGTH);
+    Integer queryLength = params.getInt(MINQUERYLENGTH);
     if (queryLength != null)
       minQueryLength = queryLength;
     
     float maxQueryFrequency = DEFAULT_MAXQUERYFREQUENCY;
-    Float queryFreq = (Float) config.get(MAXQUERYFREQUENCY);
+    Float queryFreq = params.getFloat(MAXQUERYFREQUENCY);
     if (queryFreq != null)
       maxQueryFrequency = queryFreq;
     


[27/50] lucene-solr:jira/solr-8593: SOLR-9891: Add mkroot command to bin/solr and bin/solr.cmd

Posted by kr...@apache.org.
SOLR-9891: Add mkroot command to bin/solr and bin/solr.cmd


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

Branch: refs/heads/jira/solr-8593
Commit: cb266d5fc775bd9d26ed7f0e68e9d0d12793f9b5
Parents: b584f9c
Author: Erick Erickson <er...@apache.org>
Authored: Thu Dec 29 17:39:48 2016 -0800
Committer: Erick Erickson <er...@apache.org>
Committed: Thu Dec 29 17:39:48 2016 -0800

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  2 +
 solr/bin/solr                                   | 26 +++++++-
 solr/bin/solr.cmd                               | 21 ++++++-
 .../src/java/org/apache/solr/util/SolrCLI.java  | 65 +++++++++++++++++++-
 4 files changed, 108 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cb266d5f/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 7f83de0..59dde90 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -204,6 +204,8 @@ New Features
 
 * SOLR-9905: Add NullStream to isolate the performance of the ExportWriter (Joel Bernstein)
 
+* SOLR-9891: Add mkroot command to bin/solr and bin/solr.cmd (Erick Erickson)
+
 Optimizations
 ----------------------
 * SOLR-9704: Facet Module / JSON Facet API: Optimize blockChildren facets that have

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cb266d5f/solr/bin/solr
----------------------------------------------------------------------
diff --git a/solr/bin/solr b/solr/bin/solr
index c1add26..fcf864b 100755
--- a/solr/bin/solr
+++ b/solr/bin/solr
@@ -493,6 +493,12 @@ function print_usage() {
     echo ""
     echo "             Only the node names are listed, not data"
     echo ""
+    echo "         mkroot makes a znode on Zookeeper with no data. Can be used to make a path of arbitrary"
+    echo "             depth but primarily intended to create a 'chroot'."
+    echo ""
+    echo "             <path>: The Zookeeper path to create. Leading slash is assumed if not present."
+    echo "                     Intermediate nodes are created as needed if not present."
+    echo ""
   fi
 } # end print_usage
 
@@ -507,6 +513,7 @@ function print_short_zk_usage() {
   echo "         solr zk rm [-r] <path> [-z zkHost]"
   echo "         solr zk mv <src> <dest> [-z zkHost]"
   echo "         solr zk ls [-r] <path> [-z zkHost]"
+  echo "         solr zk mkroot <path> [-z zkHost]"
   echo ""
 
   if [ "$1" == "" ]; then
@@ -1001,7 +1008,7 @@ if [[ "$SCRIPT_CMD" == "zk" ]]; then
   if [ $# -gt 0 ]; then
     while true; do
       case "$1" in
-        -upconfig|upconfig|-downconfig|downconfig|cp|rm|mv|ls)
+        -upconfig|upconfig|-downconfig|downconfig|cp|rm|mv|ls|mkroot)
             if [ "${1:0:1}" == "-" ]; then
               ZK_OP=${1:1}
             else
@@ -1042,7 +1049,7 @@ if [[ "$SCRIPT_CMD" == "zk" ]]; then
             shift
             break
         ;;
-        *)  # Pick up <src> <dst> or <path> params for rm, ls, cp, mv.
+        *)  # Pick up <src> <dst> or <path> params for rm, ls, cp, mv, mkroot.
             if [ "$1" == "" ]; then
               break # out-of-args, stop looping
             fi
@@ -1062,7 +1069,7 @@ if [[ "$SCRIPT_CMD" == "zk" ]]; then
   fi
 
   if [ -z "$ZK_OP" ]; then
-    print_short_zk_usage "Zookeeper operation (one of 'upconfig', 'downconfig', 'rm', 'mv', 'cp', 'ls') is required!"
+    print_short_zk_usage "Zookeeper operation (one of 'upconfig', 'downconfig', 'rm', 'mv', 'cp', 'ls', 'mkroot') is required!"
   fi
 
   if [ -z "$ZK_HOST" ]; then
@@ -1088,6 +1095,13 @@ if [[ "$SCRIPT_CMD" == "zk" ]]; then
     fi
   fi
 
+  if [[ "$ZK_OP" == "mkroot" ]]; then
+    if [[ -z "$ZK_SRC" ]]; then
+      print_short_zk_usage "<path> must be specified when using the 'mkroot' command."
+    fi
+  fi
+
+
   case "$ZK_OP" in
     upconfig)
       run_tool "$ZK_OP" -confname "$CONFIGSET_CONFNAME" -confdir "$CONFIGSET_CONFDIR" -zkHost "$ZK_HOST" -configsetsDir "$SOLR_TIP/server/solr/configsets"
@@ -1113,6 +1127,12 @@ if [[ "$SCRIPT_CMD" == "zk" ]]; then
       fi
       run_tool "$ZK_OP" -path "$ZK_SRC" -recurse "$ZK_RECURSE" -zkHost "$ZK_HOST"
     ;;
+    mkroot)
+      if [ -z "$ZK_SRC" ]; then
+        print_short_zk_usage "Zookeeper path to list must be specified when using the 'mkroot' command"
+      fi
+      run_tool "$ZK_OP" -path "$ZK_SRC" -zkHost "$ZK_HOST"
+    ;;
     *)
       print_short_zk_usage "Unrecognized Zookeeper operation $ZK_OP"
     ;;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cb266d5f/solr/bin/solr.cmd
----------------------------------------------------------------------
diff --git a/solr/bin/solr.cmd b/solr/bin/solr.cmd
index 4b6081f..04398bc 100644
--- a/solr/bin/solr.cmd
+++ b/solr/bin/solr.cmd
@@ -480,6 +480,13 @@ echo             ^<path^>: The Zookeeper path to use as the root.
 echo.
 echo             Only the node names are listed, not data
 echo.
+echo         mkroot makes a znode in Zookeeper with no data. Can be used to make a path of arbitrary
+echo                depth but primarily intended to create a 'chroot'."
+echo.
+echo             ^<path^>: The Zookeeper path to create. Leading slash is assumed if not present.
+echo                       Intermediate nodes are created as needed if not present.
+echo.
+
 goto done
 
 :zk_short_usage
@@ -492,6 +499,7 @@ echo         solr zk cp [-r] ^<src^> ^<dest^> [-z zkHost]
 echo         solr zk rm [-r] ^<path^> [-z zkHost]
 echo         solr zk mv ^<src^> ^<dest^> [-z zkHost]
 echo         solr zk ls [-r] ^<path^> [-z zkHost]
+echo         solr zk mkroot ^<path^> [-z zkHost]
 echo.
 IF "%ZK_FULL%"=="true" (
   goto zk_full_usage
@@ -1399,6 +1407,8 @@ IF "%1"=="-upconfig" (
   goto set_zk_op
 ) ELSE IF "%1"=="ls" (
   goto set_zk_op
+) ELSE IF "%1"=="mkroot" (
+  goto set_zk_op
 ) ELSE IF "%1"=="-n" (
   goto set_config_name
 ) ELSE IF "%1"=="-r" (
@@ -1561,13 +1571,22 @@ IF "!ZK_OP!"=="upconfig" (
   org.apache.solr.util.SolrCLI !ZK_OP! -zkHost !ZK_HOST! -path !ZK_SRC! -recurse !ZK_RECURSE!
 ) ELSE IF "!ZK_OP!"=="ls" (
   IF "%ZK_SRC"=="" (
-    set ERROR_MSG="Zookeeper path to remove must be specified when using the 'rm' command"
+    set ERROR_MSG="Zookeeper path to remove must be specified when using the 'ls' command"
     goto zk_short_usage
   )
   "%JAVA%" %SOLR_SSL_OPTS% %AUTHC_OPTS% %SOLR_ZK_CREDS_AND_ACLS% -Dsolr.install.dir="%SOLR_TIP%" ^
   -Dlog4j.configuration="file:%DEFAULT_SERVER_DIR%\scripts\cloud-scripts\log4j.properties" ^
   -classpath "%DEFAULT_SERVER_DIR%\solr-webapp\webapp\WEB-INF\lib\*;%DEFAULT_SERVER_DIR%\lib\ext\*" ^
   org.apache.solr.util.SolrCLI !ZK_OP! -zkHost !ZK_HOST! -path !ZK_SRC! -recurse !ZK_RECURSE!
+) ELSE IF "!ZK_OP!"=="mkroot" (
+  IF "%ZK_SRC"=="" (
+    set ERROR_MSG="Zookeeper path to create must be specified when using the 'mkroot' command"
+    goto zk_short_usage
+  )
+  "%JAVA%" %SOLR_SSL_OPTS% %AUTHC_OPTS% %SOLR_ZK_CREDS_AND_ACLS% -Dsolr.install.dir="%SOLR_TIP%" ^
+  -Dlog4j.configuration="file:%DEFAULT_SERVER_DIR%\scripts\cloud-scripts\log4j.properties" ^
+  -classpath "%DEFAULT_SERVER_DIR%\solr-webapp\webapp\WEB-INF\lib\*;%DEFAULT_SERVER_DIR%\lib\ext\*" ^
+  org.apache.solr.util.SolrCLI !ZK_OP! -zkHost !ZK_HOST! -path !ZK_SRC!
 ) ELSE (
   set ERROR_MSG="Unknown zk option !ZK_OP!"
   goto zk_short_usage

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cb266d5f/solr/core/src/java/org/apache/solr/util/SolrCLI.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/SolrCLI.java b/solr/core/src/java/org/apache/solr/util/SolrCLI.java
index 4979848..bb2d554 100644
--- a/solr/core/src/java/org/apache/solr/util/SolrCLI.java
+++ b/solr/core/src/java/org/apache/solr/util/SolrCLI.java
@@ -364,6 +364,8 @@ public class SolrCLI {
       return new ZkCpTool();
     else if ("ls".equals(toolType))
       return new ZkLsTool();
+    else if ("mkroot".equals(toolType))
+      return new ZkMkrootTool();
     else if ("assert".equals(toolType))
       return new AssertTool();
     else if ("utils".equals(toolType))
@@ -1986,7 +1988,7 @@ public class SolrCLI {
 
       if (zkHost == null) {
         throw new IllegalStateException("Solr at " + cli.getOptionValue("zkHost") +
-            " is running in standalone server mode, 'zk rm' can only be used when running in SolrCloud mode.\n");
+            " is running in standalone server mode, 'zk ls' can only be used when running in SolrCloud mode.\n");
       }
 
 
@@ -1999,12 +2001,71 @@ public class SolrCLI {
             " recurse: " + Boolean.toString(recurse));
         stdout.print(zkClient.listZnode(znode, recurse));
       } catch (Exception e) {
-        log.error("Could not complete rm operation for reason: " + e.getMessage());
+        log.error("Could not complete ls operation for reason: " + e.getMessage());
         throw (e);
       }
     }
   } // End zkLsTool class
 
+
+  public static class ZkMkrootTool extends ToolBase {
+
+    public ZkMkrootTool() {
+      this(System.out);
+    }
+
+    public ZkMkrootTool(PrintStream stdout) {
+      super(stdout);
+    }
+
+    @SuppressWarnings("static-access")
+    public Option[] getOptions() {
+      return new Option[]{
+          OptionBuilder
+              .withArgName("path")
+              .hasArg()
+              .isRequired(true)
+              .withDescription("Path to create")
+              .create("path"),
+          OptionBuilder
+              .withArgName("HOST")
+              .hasArg()
+              .isRequired(true)
+              .withDescription("Address of the Zookeeper ensemble; defaults to: " + ZK_HOST)
+              .create("zkHost")
+      };
+    }
+
+    public String getName() {
+      return "mkroot";
+    }
+
+    protected void runImpl(CommandLine cli) throws Exception {
+
+      String zkHost = getZkHost(cli);
+
+      if (zkHost == null) {
+        throw new IllegalStateException("Solr at " + cli.getOptionValue("zkHost") +
+            " is running in standalone server mode, 'zk mkroot' can only be used when running in SolrCloud mode.\n");
+      }
+
+
+      try (SolrZkClient zkClient = new SolrZkClient(zkHost, 30000)) {
+        echo("\nConnecting to ZooKeeper at " + zkHost + " ...");
+
+        String znode = cli.getOptionValue("path");
+        echo("Creating Zookeeper path " + znode + " on ZooKeeper at " + zkHost);
+        zkClient.makePath(znode, true);
+      } catch (Exception e) {
+        log.error("Could not complete mkroot operation for reason: " + e.getMessage());
+        throw (e);
+      }
+    }
+  } // End zkMkrootTool class
+
+
+
+
   public static class ZkCpTool extends ToolBase {
 
     public ZkCpTool() {


[41/50] lucene-solr:jira/solr-8593: Support Graph Token Streams in QueryBuilder

Posted by kr...@apache.org.
Support Graph Token Streams in QueryBuilder

Adds support for handling graph token streams inside the
QueryBuilder util class used by query parsers.


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

Branch: refs/heads/jira/solr-8593
Commit: a81ebce804947685b86b50f7525335120fde38b4
Parents: d565238
Author: Matt Weber <ma...@mattweber.org>
Authored: Mon Dec 26 07:50:58 2016 -0800
Committer: Mike McCandless <mi...@apache.org>
Committed: Tue Jan 3 05:12:33 2017 -0500

----------------------------------------------------------------------
 .../org/apache/lucene/search/GraphQuery.java    | 136 +++++++++++
 .../org/apache/lucene/util/QueryBuilder.java    | 113 +++++++--
 .../graph/GraphTokenStreamFiniteStrings.java    | 230 +++++++++++++++++++
 .../apache/lucene/search/TestGraphQuery.java    |  79 +++++++
 .../apache/lucene/util/TestQueryBuilder.java    |  15 +-
 .../TestGraphTokenStreamFiniteStrings.java      | 217 +++++++++++++++++
 .../queryparser/classic/QueryParserBase.java    |  42 +++-
 .../classic/TestMultiFieldQueryParser.java      |  11 +-
 .../queryparser/classic/TestQueryParser.java    | 131 ++++++-----
 9 files changed, 877 insertions(+), 97 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a81ebce8/lucene/core/src/java/org/apache/lucene/search/GraphQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/GraphQuery.java b/lucene/core/src/java/org/apache/lucene/search/GraphQuery.java
new file mode 100644
index 0000000..a1308c9
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/GraphQuery.java
@@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.lucene.index.IndexReader;
+
+/**
+ * A query that wraps multiple sub-queries generated from a graph token stream.
+ */
+public final class GraphQuery extends Query {
+  private final Query[] queries;
+  private boolean hasBoolean = false;
+  private boolean hasPhrase = false;
+
+  /**
+   * Constructor sets the queries and checks if any of them are
+   * a boolean query.
+   *
+   * @param queries the non-null array of queries
+   */
+  public GraphQuery(Query... queries) {
+    this.queries = Objects.requireNonNull(queries).clone();
+    for (Query query : queries) {
+      if (query instanceof BooleanQuery) {
+        hasBoolean = true;
+      } else if (query instanceof PhraseQuery) {
+        hasPhrase = true;
+      }
+    }
+  }
+
+  /**
+   * Gets the queries
+   *
+   * @return unmodifiable list of Query
+   */
+  public List<Query> getQueries() {
+    return Collections.unmodifiableList(Arrays.asList(queries));
+  }
+
+  /**
+   * If there is at least one boolean query or not.
+   *
+   * @return true if there is a boolean, false if not
+   */
+  public boolean hasBoolean() {
+    return hasBoolean;
+  }
+
+  /**
+   * If there is at least one phrase query or not.
+   *
+   * @return true if there is a phrase query, false if not
+   */
+  public boolean hasPhrase() {
+    return hasPhrase;
+  }
+
+  /**
+   * Rewrites to a single query or a boolean query where each query is a SHOULD clause.
+   */
+  @Override
+  public Query rewrite(IndexReader reader) throws IOException {
+    if (queries.length == 0) {
+      return new BooleanQuery.Builder().build();
+    }
+
+    if (queries.length == 1) {
+      return queries[0];
+    }
+
+    BooleanQuery.Builder q = new BooleanQuery.Builder();
+    for (Query clause : queries) {
+      q.add(clause, BooleanClause.Occur.SHOULD);
+    }
+
+    return q.build();
+  }
+
+  @Override
+  public String toString(String field) {
+    StringBuilder builder = new StringBuilder("Graph(");
+    for (int i = 0; i < queries.length; i++) {
+      if (i != 0) {
+        builder.append(", ");
+      }
+      builder.append(Objects.toString(queries[i]));
+    }
+
+    if (queries.length > 0) {
+      builder.append(", ");
+    }
+
+    builder.append("hasBoolean=")
+        .append(hasBoolean)
+        .append(", hasPhrase=")
+        .append(hasPhrase)
+        .append(")");
+
+    return builder.toString();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    return sameClassAs(other) &&
+        hasBoolean == ((GraphQuery) other).hasBoolean &&
+        hasPhrase == ((GraphQuery) other).hasPhrase &&
+        Arrays.equals(queries, ((GraphQuery) other).queries);
+  }
+
+  @Override
+  public int hashCode() {
+    return 31 * classHash() + Arrays.deepHashCode(new Object[]{hasBoolean, hasPhrase, queries});
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a81ebce8/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java b/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java
index 6c5ea15..a8c0a82 100644
--- a/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java
+++ b/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java
@@ -25,15 +25,18 @@ import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.CachingTokenFilter;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
 import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.GraphQuery;
 import org.apache.lucene.search.MultiPhraseQuery;
 import org.apache.lucene.search.PhraseQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.SynonymQuery;
 import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.util.graph.GraphTokenStreamFiniteStrings;
 
 /**
  * Creates queries from the {@link Analyzer} chain.
@@ -135,17 +138,39 @@ public class QueryBuilder {
     
     Query query = createFieldQuery(analyzer, BooleanClause.Occur.SHOULD, field, queryText, false, 0);
     if (query instanceof BooleanQuery) {
-      BooleanQuery bq = (BooleanQuery) query;
-      BooleanQuery.Builder builder = new BooleanQuery.Builder();
-      builder.setMinimumNumberShouldMatch((int) (fraction * bq.clauses().size()));
-      for (BooleanClause clause : bq) {
-        builder.add(clause);
+      query = addMinShouldMatchToBoolean((BooleanQuery) query, fraction);
+    } else if (query instanceof GraphQuery && ((GraphQuery) query).hasBoolean()) {
+      // we have a graph query that has at least one boolean sub-query
+      // re-build and set minimum should match on each boolean found
+      List<Query> oldQueries = ((GraphQuery) query).getQueries();
+      Query[] queries = new Query[oldQueries.size()];
+      for (int i = 0; i < queries.length; i++) {
+        Query oldQuery = oldQueries.get(i);
+        if (oldQuery instanceof BooleanQuery) {
+          queries[i] = addMinShouldMatchToBoolean((BooleanQuery) oldQuery, fraction);
+        } else {
+          queries[i] = oldQuery;
+        }
       }
-      query = builder.build();
+
+      query = new GraphQuery(queries);
     }
     return query;
   }
-  
+
+  /**
+   * Rebuilds a boolean query and sets a new minimum number should match value.
+   */
+  private BooleanQuery addMinShouldMatchToBoolean(BooleanQuery query, float fraction) {
+    BooleanQuery.Builder builder = new BooleanQuery.Builder();
+    builder.setMinimumNumberShouldMatch((int) (fraction * query.clauses().size()));
+    for (BooleanClause clause : query) {
+      builder.add(clause);
+    }
+
+    return builder.build();
+  }
+
   /** 
    * Returns the analyzer. 
    * @see #setAnalyzer(Analyzer)
@@ -183,6 +208,7 @@ public class QueryBuilder {
     this.enablePositionIncrements = enable;
   }
 
+
   /**
    * Creates a query from the analysis chain.
    * <p>
@@ -192,25 +218,44 @@ public class QueryBuilder {
    * it is usually not necessary to override it in a subclass; instead, override
    * methods like {@link #newBooleanQuery}, etc., if possible.
    *
-   * @param analyzer analyzer used for this query
-   * @param operator default boolean operator used for this query
-   * @param field field to create queries against
-   * @param queryText text to be passed to the analysis chain
-   * @param quoted true if phrases should be generated when terms occur at more than one position
+   * @param analyzer   analyzer used for this query
+   * @param operator   default boolean operator used for this query
+   * @param field      field to create queries against
+   * @param queryText  text to be passed to the analysis chain
+   * @param quoted     true if phrases should be generated when terms occur at more than one position
    * @param phraseSlop slop factor for phrase/multiphrase queries
    */
   protected Query createFieldQuery(Analyzer analyzer, BooleanClause.Occur operator, String field, String queryText, boolean quoted, int phraseSlop) {
     assert operator == BooleanClause.Occur.SHOULD || operator == BooleanClause.Occur.MUST;
-    
+
     // Use the analyzer to get all the tokens, and then build an appropriate
     // query based on the analysis chain.
-    
-    try (TokenStream source = analyzer.tokenStream(field, queryText);
-         CachingTokenFilter stream = new CachingTokenFilter(source)) {
+    try (TokenStream source = analyzer.tokenStream(field, queryText)) {
+      return createFieldQuery(source, operator, field, quoted, phraseSlop);
+    } catch (IOException e) {
+      throw new RuntimeException("Error analyzing query text", e);
+    }
+  }
+
+  /**
+   * Creates a query from a token stream.
+   *
+   * @param source     the token stream to create the query from
+   * @param operator   default boolean operator used for this query
+   * @param field      field to create queries against
+   * @param quoted     true if phrases should be generated when terms occur at more than one position
+   * @param phraseSlop slop factor for phrase/multiphrase queries
+   */
+  protected Query createFieldQuery(TokenStream source, BooleanClause.Occur operator, String field, boolean quoted, int phraseSlop) {
+    assert operator == BooleanClause.Occur.SHOULD || operator == BooleanClause.Occur.MUST;
+
+    // Build an appropriate query based on the analysis chain.
+    try (CachingTokenFilter stream = new CachingTokenFilter(source)) {
       
       TermToBytesRefAttribute termAtt = stream.getAttribute(TermToBytesRefAttribute.class);
       PositionIncrementAttribute posIncAtt = stream.addAttribute(PositionIncrementAttribute.class);
-      
+      PositionLengthAttribute posLenAtt = stream.addAttribute(PositionLengthAttribute.class);
+
       if (termAtt == null) {
         return null; 
       }
@@ -221,6 +266,7 @@ public class QueryBuilder {
       int numTokens = 0;
       int positionCount = 0;
       boolean hasSynonyms = false;
+      boolean isGraph = false;
 
       stream.reset();
       while (stream.incrementToken()) {
@@ -231,6 +277,11 @@ public class QueryBuilder {
         } else {
           hasSynonyms = true;
         }
+
+        int positionLength = posLenAtt.getPositionLength();
+        if (!isGraph && positionLength > 1) {
+          isGraph = true;
+        }
       }
       
       // phase 2: based on token count, presence of synonyms, and options
@@ -241,6 +292,9 @@ public class QueryBuilder {
       } else if (numTokens == 1) {
         // single term
         return analyzeTerm(field, stream);
+      } else if (isGraph) {
+        // graph
+        return analyzeGraph(stream, operator, field, quoted, phraseSlop);
       } else if (quoted && positionCount > 1) {
         // phrase
         if (hasSynonyms) {
@@ -388,7 +442,30 @@ public class QueryBuilder {
     }
     return mpqb.build();
   }
-  
+
+  /**
+   * Creates a query from a graph token stream by extracting all the finite strings from the graph and using them to create the query.
+   */
+  protected Query analyzeGraph(TokenStream source, BooleanClause.Occur operator, String field, boolean quoted, int phraseSlop)
+      throws IOException {
+    source.reset();
+    List<TokenStream> tokenStreams = GraphTokenStreamFiniteStrings.getTokenStreams(source);
+
+    if (tokenStreams.isEmpty()) {
+      return null;
+    }
+
+    List<Query> queries = new ArrayList<>(tokenStreams.size());
+    for (TokenStream ts : tokenStreams) {
+      Query query = createFieldQuery(ts, operator, field, quoted, phraseSlop);
+      if (query != null) {
+        queries.add(query);
+      }
+    }
+
+    return new GraphQuery(queries.toArray(new Query[0]));
+  }
+
   /**
    * Builds a new BooleanQuery instance.
    * <p>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a81ebce8/lucene/core/src/java/org/apache/lucene/util/graph/GraphTokenStreamFiniteStrings.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/graph/GraphTokenStreamFiniteStrings.java b/lucene/core/src/java/org/apache/lucene/util/graph/GraphTokenStreamFiniteStrings.java
new file mode 100644
index 0000000..cec65fa
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/util/graph/GraphTokenStreamFiniteStrings.java
@@ -0,0 +1,230 @@
+/*
+ * 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.util.graph;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.BytesTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
+import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.FiniteStringsIterator;
+import org.apache.lucene.util.automaton.Operations;
+
+import static org.apache.lucene.util.automaton.Operations.DEFAULT_MAX_DETERMINIZED_STATES;
+
+/**
+ * Creates a list of {@link TokenStream} where each stream is the tokens that make up a finite string in graph token stream.  To do this,
+ * the graph token stream is converted to an {@link Automaton} and from there we use a {@link FiniteStringsIterator} to collect the various
+ * token streams for each finite string.
+ */
+public final class GraphTokenStreamFiniteStrings {
+  private final Automaton.Builder builder = new Automaton.Builder();
+  private final Map<BytesRef, Integer> termToID = new HashMap<>();
+  private final Map<Integer, BytesRef> idToTerm = new HashMap<>();
+  private final Map<Integer, Integer> idToInc = new HashMap<>();
+  private Automaton det;
+
+  private class FiniteStringsTokenStream extends TokenStream {
+    private final BytesTermAttribute termAtt = addAttribute(BytesTermAttribute.class);
+    private final PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class);
+    private final IntsRef ids;
+    private final int end;
+    private int offset;
+
+    FiniteStringsTokenStream(final IntsRef ids) {
+      assert ids != null;
+      this.ids = ids;
+      this.offset = ids.offset;
+      this.end = ids.offset + ids.length;
+    }
+
+    @Override
+    public boolean incrementToken() throws IOException {
+      if (offset < end) {
+        clearAttributes();
+        int id = ids.ints[offset];
+        termAtt.setBytesRef(idToTerm.get(id));
+
+        int incr = 1;
+        if (idToInc.containsKey(id)) {
+          incr = idToInc.get(id);
+        }
+        posIncAtt.setPositionIncrement(incr);
+        offset++;
+        return true;
+      }
+
+      return false;
+    }
+  }
+
+  private GraphTokenStreamFiniteStrings() {
+  }
+
+  /**
+   * Gets the list of finite string token streams from the given input graph token stream.
+   */
+  public static List<TokenStream> getTokenStreams(final TokenStream in) throws IOException {
+    GraphTokenStreamFiniteStrings gfs = new GraphTokenStreamFiniteStrings();
+    return gfs.process(in);
+  }
+
+  /**
+   * Builds automaton and builds the finite string token streams.
+   */
+  private List<TokenStream> process(final TokenStream in) throws IOException {
+    build(in);
+
+    List<TokenStream> tokenStreams = new ArrayList<>();
+    final FiniteStringsIterator finiteStrings = new FiniteStringsIterator(det);
+    for (IntsRef ids; (ids = finiteStrings.next()) != null; ) {
+      tokenStreams.add(new FiniteStringsTokenStream(IntsRef.deepCopyOf(ids)));
+    }
+
+    return tokenStreams;
+  }
+
+  private void build(final TokenStream in) throws IOException {
+    if (det != null) {
+      throw new IllegalStateException("Automation already built");
+    }
+
+    final TermToBytesRefAttribute termBytesAtt = in.addAttribute(TermToBytesRefAttribute.class);
+    final PositionIncrementAttribute posIncAtt = in.addAttribute(PositionIncrementAttribute.class);
+    final PositionLengthAttribute posLengthAtt = in.addAttribute(PositionLengthAttribute.class);
+
+    in.reset();
+
+    int pos = -1;
+    int prevIncr = 1;
+    int state = -1;
+    while (in.incrementToken()) {
+      int currentIncr = posIncAtt.getPositionIncrement();
+      if (pos == -1 && currentIncr < 1) {
+        throw new IllegalStateException("Malformed TokenStream, start token can't have increment less than 1");
+      }
+
+      // always use inc 1 while building, but save original increment
+      int incr = Math.min(1, currentIncr);
+      if (incr > 0) {
+        pos += incr;
+      }
+
+      int endPos = pos + posLengthAtt.getPositionLength();
+      while (state < endPos) {
+        state = createState();
+      }
+
+      BytesRef term = termBytesAtt.getBytesRef();
+      int id = getTermID(currentIncr, prevIncr, term);
+      addTransition(pos, endPos, currentIncr, id);
+
+      // only save last increment on non-zero increment in case we have multiple stacked tokens
+      if (currentIncr > 0) {
+        prevIncr = currentIncr;
+      }
+    }
+
+    in.end();
+    setAccept(state, true);
+    finish();
+  }
+
+  /**
+   * Returns a new state; state 0 is always the initial state.
+   */
+  private int createState() {
+    return builder.createState();
+  }
+
+  /**
+   * Marks the specified state as accept or not.
+   */
+  private void setAccept(int state, boolean accept) {
+    builder.setAccept(state, accept);
+  }
+
+  /**
+   * Adds a transition to the automaton.
+   */
+  private void addTransition(int source, int dest, int incr, int id) {
+    builder.addTransition(source, dest, id);
+  }
+
+  /**
+   * Call this once you are done adding states/transitions.
+   */
+  private void finish() {
+    finish(DEFAULT_MAX_DETERMINIZED_STATES);
+  }
+
+  /**
+   * Call this once you are done adding states/transitions.
+   *
+   * @param maxDeterminizedStates Maximum number of states created when determinizing the automaton.  Higher numbers allow this operation
+   *                              to consume more memory but allow more complex automatons.
+   */
+  private void finish(int maxDeterminizedStates) {
+    Automaton automaton = builder.finish();
+    det = Operations.removeDeadStates(Operations.determinize(automaton, maxDeterminizedStates));
+  }
+
+  /**
+   * Gets an integer id for a given term.
+   *
+   * If there is no position gaps for this token then we can reuse the id for the same term if it appeared at another
+   * position without a gap.  If we have a position gap generate a new id so we can keep track of the position
+   * increment.
+   */
+  private int getTermID(int incr, int prevIncr, BytesRef term) {
+    assert term != null;
+    boolean isStackedGap = incr == 0 && prevIncr > 1;
+    boolean hasGap = incr > 1;
+    Integer id;
+    if (hasGap || isStackedGap) {
+      id = idToTerm.size();
+      idToTerm.put(id, BytesRef.deepCopyOf(term));
+
+      // stacked token should have the same increment as original token at this position
+      if (isStackedGap) {
+        idToInc.put(id, prevIncr);
+      } else {
+        idToInc.put(id, incr);
+      }
+    } else {
+      id = termToID.get(term);
+      if (id == null) {
+        term = BytesRef.deepCopyOf(term);
+        id = idToTerm.size();
+        termToID.put(term, id);
+        idToTerm.put(id, term);
+      }
+    }
+
+    return id;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a81ebce8/lucene/core/src/test/org/apache/lucene/search/TestGraphQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestGraphQuery.java b/lucene/core/src/test/org/apache/lucene/search/TestGraphQuery.java
new file mode 100644
index 0000000..412fac4
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/TestGraphQuery.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search;
+
+
+import java.io.IOException;
+
+import org.apache.lucene.index.Term;
+import org.apache.lucene.util.LuceneTestCase;
+
+public class TestGraphQuery extends LuceneTestCase {
+
+  public void testEquals() {
+    QueryUtils.checkEqual(new GraphQuery(), new GraphQuery());
+    QueryUtils.checkEqual(new GraphQuery(new MatchAllDocsQuery()), new GraphQuery(new MatchAllDocsQuery()));
+    QueryUtils.checkEqual(
+        new GraphQuery(new TermQuery(new Term("a", "a")), new TermQuery(new Term("a", "b"))),
+        new GraphQuery(new TermQuery(new Term("a", "a")), new TermQuery(new Term("a", "b")))
+    );
+  }
+
+  public void testBooleanDetection() {
+    assertFalse(new GraphQuery().hasBoolean());
+    assertFalse(new GraphQuery(new MatchAllDocsQuery(), new TermQuery(new Term("a", "a"))).hasBoolean());
+    assertTrue(new GraphQuery(new BooleanQuery.Builder().build()).hasBoolean());
+    assertTrue(new GraphQuery(new TermQuery(new Term("a", "a")), new BooleanQuery.Builder().build()).hasBoolean());
+  }
+
+  public void testPhraseDetection() {
+    assertFalse(new GraphQuery().hasPhrase());
+    assertFalse(new GraphQuery(new MatchAllDocsQuery(), new TermQuery(new Term("a", "a"))).hasPhrase());
+    assertTrue(new GraphQuery(new PhraseQuery.Builder().build()).hasPhrase());
+    assertTrue(new GraphQuery(new TermQuery(new Term("a", "a")), new PhraseQuery.Builder().build()).hasPhrase());
+  }
+
+  public void testToString() {
+    assertEquals("Graph(hasBoolean=false, hasPhrase=false)", new GraphQuery().toString());
+    assertEquals("Graph(a:a, a:b, hasBoolean=true, hasPhrase=false)",
+        new GraphQuery(new TermQuery(new Term("a", "a")),
+            new BooleanQuery.Builder().add(new TermQuery(new Term("a", "b")), BooleanClause.Occur.SHOULD)
+                .build()).toString());
+    assertEquals("Graph(a:\"a b\", a:b, hasBoolean=true, hasPhrase=true)",
+        new GraphQuery(
+            new PhraseQuery.Builder()
+                .add(new Term("a", "a"))
+                .add(new Term("a", "b")).build(),
+            new BooleanQuery.Builder().add(new TermQuery(new Term("a", "b")), BooleanClause.Occur.SHOULD)
+                .build()).toString());
+  }
+
+  public void testRewrite() throws IOException {
+    QueryUtils.checkEqual(new BooleanQuery.Builder().build(), new GraphQuery().rewrite(null));
+    QueryUtils.checkEqual(new TermQuery(new Term("a", "a")),
+        new GraphQuery(new TermQuery(new Term("a", "a"))).rewrite(null));
+    QueryUtils.checkEqual(
+        new BooleanQuery.Builder()
+            .add(new TermQuery(new Term("a", "a")), BooleanClause.Occur.SHOULD)
+            .add(new TermQuery(new Term("b", "b")), BooleanClause.Occur.SHOULD).build(),
+        new GraphQuery(
+            new TermQuery(new Term("a", "a")),
+            new TermQuery(new Term("b", "b"))
+        ).rewrite(null)
+    );
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a81ebce8/lucene/core/src/test/org/apache/lucene/util/TestQueryBuilder.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/TestQueryBuilder.java b/lucene/core/src/test/org/apache/lucene/util/TestQueryBuilder.java
index d3019e3..9cd8390 100644
--- a/lucene/core/src/test/org/apache/lucene/util/TestQueryBuilder.java
+++ b/lucene/core/src/test/org/apache/lucene/util/TestQueryBuilder.java
@@ -31,6 +31,7 @@ import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.GraphQuery;
 import org.apache.lucene.search.MultiPhraseQuery;
 import org.apache.lucene.search.PhraseQuery;
 import org.apache.lucene.search.Query;
@@ -150,13 +151,17 @@ public class TestQueryBuilder extends LuceneTestCase {
     assertEquals(expectedBuilder.build(), builder.createPhraseQuery("field", "old dogs"));
   }
 
-  /** forms multiphrase query */
+  /** forms graph query */
   public void testMultiWordSynonymsPhrase() throws Exception {
-    MultiPhraseQuery.Builder expectedBuilder = new MultiPhraseQuery.Builder();
-    expectedBuilder.add(new Term[] { new Term("field", "guinea"), new Term("field", "cavy") });
-    expectedBuilder.add(new Term("field", "pig"));
+    PhraseQuery.Builder expectedPhrase = new PhraseQuery.Builder();
+    expectedPhrase.add(new Term("field", "guinea"));
+    expectedPhrase.add(new Term("field", "pig"));
+
+    TermQuery expectedTerm = new TermQuery(new Term("field", "cavy"));
+
     QueryBuilder queryBuilder = new QueryBuilder(new MockSynonymAnalyzer());
-    assertEquals(expectedBuilder.build(), queryBuilder.createPhraseQuery("field", "guinea pig"));
+    assertEquals(new GraphQuery(expectedPhrase.build(), expectedTerm),
+        queryBuilder.createPhraseQuery("field", "guinea pig"));
   }
 
   protected static class SimpleCJKTokenizer extends Tokenizer {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a81ebce8/lucene/core/src/test/org/apache/lucene/util/graph/TestGraphTokenStreamFiniteStrings.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/graph/TestGraphTokenStreamFiniteStrings.java b/lucene/core/src/test/org/apache/lucene/util/graph/TestGraphTokenStreamFiniteStrings.java
new file mode 100644
index 0000000..4e636e2
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/util/graph/TestGraphTokenStreamFiniteStrings.java
@@ -0,0 +1,217 @@
+/*
+ * 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.util.graph;
+
+import java.util.List;
+
+import org.apache.lucene.analysis.CannedTokenStream;
+import org.apache.lucene.analysis.Token;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.BytesTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.util.LuceneTestCase;
+
+/**
+ * {@link GraphTokenStreamFiniteStrings} tests.
+ */
+public class TestGraphTokenStreamFiniteStrings extends LuceneTestCase {
+
+  private static Token token(String term, int posInc, int posLength) {
+    final Token t = new Token(term, 0, term.length());
+    t.setPositionIncrement(posInc);
+    t.setPositionLength(posLength);
+    return t;
+  }
+
+  private void assertTokenStream(TokenStream ts, String[] terms, int[] increments) throws Exception {
+    // verify no nulls and arrays same length
+    assertNotNull(ts);
+    assertNotNull(terms);
+    assertNotNull(increments);
+    assertEquals(terms.length, increments.length);
+    BytesTermAttribute termAtt = ts.getAttribute(BytesTermAttribute.class);
+    PositionIncrementAttribute incrAtt = ts.getAttribute(PositionIncrementAttribute.class);
+    int offset = 0;
+    while (ts.incrementToken()) {
+      // verify term and increment
+      assert offset < terms.length;
+      assertEquals(terms[offset], termAtt.getBytesRef().utf8ToString());
+      assertEquals(increments[offset], incrAtt.getPositionIncrement());
+      offset++;
+    }
+
+    // make sure we processed all items
+    assertEquals(offset, terms.length);
+  }
+
+  public void testIllegalState() throws Exception {
+    expectThrows(IllegalStateException.class, () -> {
+      TokenStream ts = new CannedTokenStream(
+          token("a", 0, 1),
+          token("b", 1, 1)
+      );
+
+      GraphTokenStreamFiniteStrings.getTokenStreams(ts);
+    });
+  }
+
+  public void testSingleGraph() throws Exception {
+    TokenStream ts = new CannedTokenStream(
+        token("fast", 1, 1),
+        token("wi", 1, 1),
+        token("wifi", 0, 2),
+        token("fi", 1, 1),
+        token("network", 1, 1)
+    );
+
+    List<TokenStream> finiteTokenStreams = GraphTokenStreamFiniteStrings.getTokenStreams(ts);
+
+    assertEquals(2, finiteTokenStreams.size());
+    assertTokenStream(finiteTokenStreams.get(0), new String[]{"fast", "wi", "fi", "network"}, new int[]{1, 1, 1, 1});
+    assertTokenStream(finiteTokenStreams.get(1), new String[]{"fast", "wifi", "network"}, new int[]{1, 1, 1});
+  }
+
+  public void testSingleGraphWithGap() throws Exception {
+    // "hey the fast wifi network", where "the" removed
+    TokenStream ts = new CannedTokenStream(
+        token("hey", 1, 1),
+        token("fast", 2, 1),
+        token("wi", 1, 1),
+        token("wifi", 0, 2),
+        token("fi", 1, 1),
+        token("network", 1, 1)
+    );
+
+    List<TokenStream> finiteTokenStreams = GraphTokenStreamFiniteStrings.getTokenStreams(ts);
+
+    assertEquals(2, finiteTokenStreams.size());
+    assertTokenStream(finiteTokenStreams.get(0),
+        new String[]{"hey", "fast", "wi", "fi", "network"}, new int[]{1, 2, 1, 1, 1});
+    assertTokenStream(finiteTokenStreams.get(1),
+        new String[]{"hey", "fast", "wifi", "network"}, new int[]{1, 2, 1, 1});
+  }
+
+
+  public void testGraphAndGapSameToken() throws Exception {
+    TokenStream ts = new CannedTokenStream(
+        token("fast", 1, 1),
+        token("wi", 2, 1),
+        token("wifi", 0, 2),
+        token("fi", 1, 1),
+        token("network", 1, 1)
+    );
+
+    List<TokenStream> finiteTokenStreams = GraphTokenStreamFiniteStrings.getTokenStreams(ts);
+
+    assertEquals(2, finiteTokenStreams.size());
+    assertTokenStream(finiteTokenStreams.get(0), new String[]{"fast", "wi", "fi", "network"}, new int[]{1, 2, 1, 1});
+    assertTokenStream(finiteTokenStreams.get(1), new String[]{"fast", "wifi", "network"}, new int[]{1, 2, 1});
+  }
+
+  public void testGraphAndGapSameTokenTerm() throws Exception {
+    TokenStream ts = new CannedTokenStream(
+        token("a", 1, 1),
+        token("b", 1, 1),
+        token("c", 2, 1),
+        token("a", 0, 2),
+        token("d", 1, 1)
+    );
+
+    List<TokenStream> finiteTokenStreams = GraphTokenStreamFiniteStrings.getTokenStreams(ts);
+
+    assertEquals(2, finiteTokenStreams.size());
+    assertTokenStream(finiteTokenStreams.get(0), new String[]{"a", "b", "c", "d"}, new int[]{1, 1, 2, 1});
+    assertTokenStream(finiteTokenStreams.get(1), new String[]{"a", "b", "a"}, new int[]{1, 1, 2});
+  }
+
+  public void testStackedGraph() throws Exception {
+    TokenStream ts = new CannedTokenStream(
+        token("fast", 1, 1),
+        token("wi", 1, 1),
+        token("wifi", 0, 2),
+        token("wireless", 0, 2),
+        token("fi", 1, 1),
+        token("network", 1, 1)
+    );
+
+    List<TokenStream> finiteTokenStreams = GraphTokenStreamFiniteStrings.getTokenStreams(ts);
+
+    assertEquals(3, finiteTokenStreams.size());
+    assertTokenStream(finiteTokenStreams.get(0), new String[]{"fast", "wi", "fi", "network"}, new int[]{1, 1, 1, 1});
+    assertTokenStream(finiteTokenStreams.get(1), new String[]{"fast", "wifi", "network"}, new int[]{1, 1, 1});
+    assertTokenStream(finiteTokenStreams.get(2), new String[]{"fast", "wireless", "network"}, new int[]{1, 1, 1});
+  }
+
+  public void testStackedGraphWithGap() throws Exception {
+    TokenStream ts = new CannedTokenStream(
+        token("fast", 1, 1),
+        token("wi", 2, 1),
+        token("wifi", 0, 2),
+        token("wireless", 0, 2),
+        token("fi", 1, 1),
+        token("network", 1, 1)
+    );
+
+    List<TokenStream> finiteTokenStreams = GraphTokenStreamFiniteStrings.getTokenStreams(ts);
+
+    assertEquals(3, finiteTokenStreams.size());
+    assertTokenStream(finiteTokenStreams.get(0), new String[]{"fast", "wi", "fi", "network"}, new int[]{1, 2, 1, 1});
+    assertTokenStream(finiteTokenStreams.get(1), new String[]{"fast", "wifi", "network"}, new int[]{1, 2, 1});
+    assertTokenStream(finiteTokenStreams.get(2), new String[]{"fast", "wireless", "network"}, new int[]{1, 2, 1});
+  }
+
+  public void testGraphWithRegularSynonym() throws Exception {
+    TokenStream ts = new CannedTokenStream(
+        token("fast", 1, 1),
+        token("speedy", 0, 1),
+        token("wi", 1, 1),
+        token("wifi", 0, 2),
+        token("fi", 1, 1),
+        token("network", 1, 1)
+    );
+
+    List<TokenStream> finiteTokenStreams = GraphTokenStreamFiniteStrings.getTokenStreams(ts);
+
+    assertEquals(4, finiteTokenStreams.size());
+    assertTokenStream(finiteTokenStreams.get(0), new String[]{"fast", "wi", "fi", "network"}, new int[]{1, 1, 1, 1});
+    assertTokenStream(finiteTokenStreams.get(1), new String[]{"fast", "wifi", "network"}, new int[]{1, 1, 1});
+    assertTokenStream(finiteTokenStreams.get(2), new String[]{"speedy", "wi", "fi", "network"}, new int[]{1, 1, 1, 1});
+    assertTokenStream(finiteTokenStreams.get(3), new String[]{"speedy", "wifi", "network"}, new int[]{1, 1, 1});
+  }
+
+  public void testMultiGraph() throws Exception {
+    TokenStream ts = new CannedTokenStream(
+        token("turbo", 1, 1),
+        token("fast", 0, 2),
+        token("charged", 1, 1),
+        token("wi", 1, 1),
+        token("wifi", 0, 2),
+        token("fi", 1, 1),
+        token("network", 1, 1)
+    );
+
+    List<TokenStream> finiteTokenStreams = GraphTokenStreamFiniteStrings.getTokenStreams(ts);
+
+    assertEquals(4, finiteTokenStreams.size());
+    assertTokenStream(finiteTokenStreams.get(0),
+        new String[]{"turbo", "charged", "wi", "fi", "network"}, new int[]{1, 1, 1, 1, 1});
+    assertTokenStream(finiteTokenStreams.get(1),
+        new String[]{"turbo", "charged", "wifi", "network"}, new int[]{1, 1, 1, 1});
+    assertTokenStream(finiteTokenStreams.get(2), new String[]{"fast", "wi", "fi", "network"}, new int[]{1, 1, 1, 1});
+    assertTokenStream(finiteTokenStreams.get(3), new String[]{"fast", "wifi", "network"}, new int[]{1, 1, 1});
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a81ebce8/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java
index 41d3764..9b238d87 100644
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java
@@ -475,8 +475,6 @@ public abstract class QueryParserBase extends QueryBuilder implements CommonQuer
     return createFieldQuery(analyzer, occur, field, queryText, quoted || autoGeneratePhraseQueries, phraseSlop);
   }
 
-
-
   /**
    * Base implementation delegates to {@link #getFieldQuery(String,String,boolean)}.
    * This method may be overridden, for example, to return
@@ -489,26 +487,48 @@ public abstract class QueryParserBase extends QueryBuilder implements CommonQuer
     Query query = getFieldQuery(field, queryText, true);
 
     if (query instanceof PhraseQuery) {
-      PhraseQuery.Builder builder = new PhraseQuery.Builder();
-      builder.setSlop(slop);
-      PhraseQuery pq = (PhraseQuery) query;
-      org.apache.lucene.index.Term[] terms = pq.getTerms();
-      int[] positions = pq.getPositions();
-      for (int i = 0; i < terms.length; ++i) {
-        builder.add(terms[i], positions[i]);
-      }
-      query = builder.build();
+      query = addSlopToPhrase((PhraseQuery) query, slop);
     } else if (query instanceof MultiPhraseQuery) {
       MultiPhraseQuery mpq = (MultiPhraseQuery)query;
       
       if (slop != mpq.getSlop()) {
         query = new MultiPhraseQuery.Builder(mpq).setSlop(slop).build();
       }
+    } else if (query instanceof GraphQuery && ((GraphQuery) query).hasPhrase()) {
+      // we have a graph query that has at least one phrase sub-query
+      // re-build and set slop on all phrase queries
+      List<Query> oldQueries = ((GraphQuery) query).getQueries();
+      Query[] queries = new Query[oldQueries.size()];
+      for (int i = 0; i < queries.length; i++) {
+        Query oldQuery = oldQueries.get(i);
+        if (oldQuery instanceof PhraseQuery) {
+          queries[i] = addSlopToPhrase((PhraseQuery) oldQuery, slop);
+        } else {
+          queries[i] = oldQuery;
+        }
+      }
+
+      query = new GraphQuery(queries);
     }
 
     return query;
   }
 
+  /**
+   * Rebuild a phrase query with a slop value
+   */
+  private PhraseQuery addSlopToPhrase(PhraseQuery query, int slop) {
+    PhraseQuery.Builder builder = new PhraseQuery.Builder();
+    builder.setSlop(slop);
+    org.apache.lucene.index.Term[] terms = query.getTerms();
+    int[] positions = query.getPositions();
+    for (int i = 0; i < terms.length; ++i) {
+      builder.add(terms[i], positions[i]);
+    }
+
+    return builder.build();
+  }
+
   protected Query getRangeQuery(String field,
                                 String part1,
                                 String part2,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a81ebce8/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestMultiFieldQueryParser.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestMultiFieldQueryParser.java b/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestMultiFieldQueryParser.java
index ed76ff5..ae15284 100644
--- a/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestMultiFieldQueryParser.java
+++ b/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestMultiFieldQueryParser.java
@@ -21,15 +21,19 @@ import java.io.StringReader;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.lucene.analysis.*;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.analysis.MockSynonymFilter;
+import org.apache.lucene.analysis.MockTokenizer;
+import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
@@ -347,7 +351,8 @@ public class TestMultiFieldQueryParser extends LuceneTestCase {
     assertEquals("Synonym(b:dog b:dogs) Synonym(t:dog t:dogs)", q.toString());
     q = parser.parse("guinea pig");
     assertFalse(parser.getSplitOnWhitespace());
-    assertEquals("(Synonym(b:cavy b:guinea) Synonym(t:cavy t:guinea)) (b:pig t:pig)", q.toString());
+    assertEquals("Graph(b:guinea b:pig, b:cavy, hasBoolean=true, hasPhrase=false) "
+        + "Graph(t:guinea t:pig, t:cavy, hasBoolean=true, hasPhrase=false)", q.toString());
     parser.setSplitOnWhitespace(true);
     q = parser.parse("guinea pig");
     assertEquals("(b:guinea t:guinea) (b:pig t:pig)", q.toString());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a81ebce8/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestQueryParser.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestQueryParser.java b/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestQueryParser.java
index bb97624..87bc89f 100644
--- a/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestQueryParser.java
+++ b/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestQueryParser.java
@@ -16,6 +16,8 @@
  */
 package org.apache.lucene.queryparser.classic;
 
+import java.io.IOException;
+
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockBytesAnalyzer;
@@ -27,10 +29,10 @@ import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.document.DateTools.Resolution;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
-import org.apache.lucene.document.DateTools.Resolution;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.RandomIndexWriter;
@@ -41,6 +43,7 @@ import org.apache.lucene.queryparser.util.QueryParserTestBase;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.BoostQuery;
+import org.apache.lucene.search.GraphQuery;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.MultiPhraseQuery;
@@ -51,8 +54,6 @@ import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.automaton.TooComplexToDeterminizeException;
 
-import java.io.IOException;
-
 /**
  * Tests QueryParser.
  */
@@ -502,32 +503,34 @@ public class TestQueryParser extends QueryParserTestBase {
     QueryParser dumb = new QueryParser("field", new Analyzer1());
     dumb.setSplitOnWhitespace(false);
 
-    // A multi-word synonym source will form a synonym query for the same-starting-position tokens
-    BooleanQuery.Builder multiWordExpandedBqBuilder = new BooleanQuery.Builder();
-    Query multiWordSynonymQuery = new SynonymQuery(new Term("field", "guinea"), new Term("field", "cavy"));
-    multiWordExpandedBqBuilder.add(multiWordSynonymQuery, BooleanClause.Occur.SHOULD);
-    multiWordExpandedBqBuilder.add(new TermQuery(new Term("field", "pig")), BooleanClause.Occur.SHOULD);
-    Query multiWordExpandedBq = multiWordExpandedBqBuilder.build();
-    assertEquals(multiWordExpandedBq, dumb.parse("guinea pig"));
-
-    // With the phrase operator, a multi-word synonym source will form a multiphrase query.
-    // When the number of expanded term(s) is different from that of the original term(s), this is not good.
-    MultiPhraseQuery.Builder multiWordExpandedMpqBuilder = new MultiPhraseQuery.Builder();
-    multiWordExpandedMpqBuilder.add(new Term[]{new Term("field", "guinea"), new Term("field", "cavy")});
-    multiWordExpandedMpqBuilder.add(new Term("field", "pig"));
-    Query multiWordExpandedMPQ = multiWordExpandedMpqBuilder.build();
-    assertEquals(multiWordExpandedMPQ, dumb.parse("\"guinea pig\""));
+    TermQuery guinea = new TermQuery(new Term("field", "guinea"));
+    TermQuery pig = new TermQuery(new Term("field", "pig"));
+    TermQuery cavy = new TermQuery(new Term("field", "cavy"));
+
+    // A multi-word synonym source will form a graph query for synonyms that formed the graph token stream
+    BooleanQuery.Builder synonym = new BooleanQuery.Builder();
+    synonym.add(guinea, BooleanClause.Occur.SHOULD);
+    synonym.add(pig, BooleanClause.Occur.SHOULD);
+    BooleanQuery guineaPig = synonym.build();
+
+    GraphQuery graphQuery = new GraphQuery(guineaPig, cavy);
+    assertEquals(graphQuery, dumb.parse("guinea pig"));
+
+    // With the phrase operator, a multi-word synonym source will form a graph query with inner phrase queries.
+    PhraseQuery.Builder phraseSynonym = new PhraseQuery.Builder();
+    phraseSynonym.add(new Term("field", "guinea"));
+    phraseSynonym.add(new Term("field", "pig"));
+    PhraseQuery guineaPigPhrase = phraseSynonym.build();
+
+    graphQuery = new GraphQuery(guineaPigPhrase, cavy);
+    assertEquals(graphQuery, dumb.parse("\"guinea pig\""));
 
     // custom behavior, the synonyms are expanded, unless you use quote operator
     QueryParser smart = new SmartQueryParser();
     smart.setSplitOnWhitespace(false);
-    assertEquals(multiWordExpandedBq, smart.parse("guinea pig"));
-
-    PhraseQuery.Builder multiWordUnexpandedPqBuilder = new PhraseQuery.Builder();
-    multiWordUnexpandedPqBuilder.add(new Term("field", "guinea"));
-    multiWordUnexpandedPqBuilder.add(new Term("field", "pig"));
-    Query multiWordUnexpandedPq = multiWordUnexpandedPqBuilder.build();
-    assertEquals(multiWordUnexpandedPq, smart.parse("\"guinea pig\""));
+    graphQuery = new GraphQuery(guineaPig, cavy);
+    assertEquals(graphQuery, smart.parse("guinea pig"));
+    assertEquals(guineaPigPhrase, smart.parse("\"guinea pig\""));
   }
 
   // TODO: Move to QueryParserTestBase once standard flexible parser gets this capability
@@ -580,34 +583,34 @@ public class TestQueryParser extends QueryParserTestBase {
     assertQueryEquals("guinea /pig/", a, "guinea /pig/");
 
     // Operators should not interrupt multiword analysis if not don't associate
-    assertQueryEquals("(guinea pig)", a, "Synonym(cavy guinea) pig");
-    assertQueryEquals("+(guinea pig)", a, "+(Synonym(cavy guinea) pig)");
-    assertQueryEquals("-(guinea pig)", a, "-(Synonym(cavy guinea) pig)");
-    assertQueryEquals("!(guinea pig)", a, "-(Synonym(cavy guinea) pig)");
-    assertQueryEquals("NOT (guinea pig)", a, "-(Synonym(cavy guinea) pig)");
-    assertQueryEquals("(guinea pig)^2", a, "(Synonym(cavy guinea) pig)^2.0");
-
-    assertQueryEquals("field:(guinea pig)", a, "Synonym(cavy guinea) pig");
-
-    assertQueryEquals("+small guinea pig", a, "+small Synonym(cavy guinea) pig");
-    assertQueryEquals("-small guinea pig", a, "-small Synonym(cavy guinea) pig");
-    assertQueryEquals("!small guinea pig", a, "-small Synonym(cavy guinea) pig");
-    assertQueryEquals("NOT small guinea pig", a, "-small Synonym(cavy guinea) pig");
-    assertQueryEquals("small* guinea pig", a, "small* Synonym(cavy guinea) pig");
-    assertQueryEquals("small? guinea pig", a, "small? Synonym(cavy guinea) pig");
-    assertQueryEquals("\"small\" guinea pig", a, "small Synonym(cavy guinea) pig");
-
-    assertQueryEquals("guinea pig +running", a, "Synonym(cavy guinea) pig +running");
-    assertQueryEquals("guinea pig -running", a, "Synonym(cavy guinea) pig -running");
-    assertQueryEquals("guinea pig !running", a, "Synonym(cavy guinea) pig -running");
-    assertQueryEquals("guinea pig NOT running", a, "Synonym(cavy guinea) pig -running");
-    assertQueryEquals("guinea pig running*", a, "Synonym(cavy guinea) pig running*");
-    assertQueryEquals("guinea pig running?", a, "Synonym(cavy guinea) pig running?");
-    assertQueryEquals("guinea pig \"running\"", a, "Synonym(cavy guinea) pig running");
-
-    assertQueryEquals("\"guinea pig\"~2", a, "\"(guinea cavy) pig\"~2");
-
-    assertQueryEquals("field:\"guinea pig\"", a, "\"(guinea cavy) pig\"");
+    assertQueryEquals("(guinea pig)", a, "Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
+    assertQueryEquals("+(guinea pig)", a, "+Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
+    assertQueryEquals("-(guinea pig)", a, "-Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
+    assertQueryEquals("!(guinea pig)", a, "-Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
+    assertQueryEquals("NOT (guinea pig)", a, "-Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
+    assertQueryEquals("(guinea pig)^2", a, "(Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false))^2.0");
+
+    assertQueryEquals("field:(guinea pig)", a, "Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
+
+    assertQueryEquals("+small guinea pig", a, "+small Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
+    assertQueryEquals("-small guinea pig", a, "-small Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
+    assertQueryEquals("!small guinea pig", a, "-small Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
+    assertQueryEquals("NOT small guinea pig", a, "-small Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
+    assertQueryEquals("small* guinea pig", a, "small* Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
+    assertQueryEquals("small? guinea pig", a, "small? Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
+    assertQueryEquals("\"small\" guinea pig", a, "small Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
+
+    assertQueryEquals("guinea pig +running", a, "Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false) +running");
+    assertQueryEquals("guinea pig -running", a, "Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false) -running");
+    assertQueryEquals("guinea pig !running", a, "Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false) -running");
+    assertQueryEquals("guinea pig NOT running", a, "Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false) -running");
+    assertQueryEquals("guinea pig running*", a, "Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false) running*");
+    assertQueryEquals("guinea pig running?", a, "Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false) running?");
+    assertQueryEquals("guinea pig \"running\"", a, "Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false) running");
+
+    assertQueryEquals("\"guinea pig\"~2", a, "Graph(field:\"guinea pig\"~2, field:cavy, hasBoolean=false, hasPhrase=true)");
+
+    assertQueryEquals("field:\"guinea pig\"", a, "Graph(field:\"guinea pig\", field:cavy, hasBoolean=false, hasPhrase=true)");
 
     splitOnWhitespace = oldSplitOnWhitespace;
   }
@@ -684,9 +687,9 @@ public class TestQueryParser extends QueryParserTestBase {
     assertQueryEquals("guinea pig running?", a, "guinea pig running?");
     assertQueryEquals("guinea pig \"running\"", a, "guinea pig running");
 
-    assertQueryEquals("\"guinea pig\"~2", a, "\"(guinea cavy) pig\"~2");
+    assertQueryEquals("\"guinea pig\"~2", a, "Graph(field:\"guinea pig\"~2, field:cavy, hasBoolean=false, hasPhrase=true)");
 
-    assertQueryEquals("field:\"guinea pig\"", a, "\"(guinea cavy) pig\"");
+    assertQueryEquals("field:\"guinea pig\"", a, "Graph(field:\"guinea pig\", field:cavy, hasBoolean=false, hasPhrase=true)");
 
     splitOnWhitespace = oldSplitOnWhitespace;
   }
@@ -697,14 +700,22 @@ public class TestQueryParser extends QueryParserTestBase {
     assertFalse(parser.getSplitOnWhitespace()); // default is false
 
     // A multi-word synonym source will form a synonym query for the same-starting-position tokens
-    BooleanQuery.Builder bqBuilder = new BooleanQuery.Builder();
-    bqBuilder.add(new SynonymQuery(new Term("field", "guinea"), new Term("field", "cavy")), BooleanClause.Occur.SHOULD);
-    bqBuilder.add(new TermQuery(new Term("field", "pig")), BooleanClause.Occur.SHOULD);
-    assertEquals(bqBuilder.build(), parser.parse("guinea pig"));
+    TermQuery guinea = new TermQuery(new Term("field", "guinea"));
+    TermQuery pig = new TermQuery(new Term("field", "pig"));
+    TermQuery cavy = new TermQuery(new Term("field", "cavy"));
+
+    // A multi-word synonym source will form a graph query for synonyms that formed the graph token stream
+    BooleanQuery.Builder synonym = new BooleanQuery.Builder();
+    synonym.add(guinea, BooleanClause.Occur.SHOULD);
+    synonym.add(pig, BooleanClause.Occur.SHOULD);
+    BooleanQuery guineaPig = synonym.build();
+
+    GraphQuery graphQuery = new GraphQuery(guineaPig, cavy);
+    assertEquals(graphQuery, parser.parse("guinea pig"));
 
     boolean oldSplitOnWhitespace = splitOnWhitespace;
     splitOnWhitespace = QueryParser.DEFAULT_SPLIT_ON_WHITESPACE;
-    assertQueryEquals("guinea pig", new MockSynonymAnalyzer(), "Synonym(cavy guinea) pig");
+    assertQueryEquals("guinea pig", new MockSynonymAnalyzer(), "Graph(field:guinea field:pig, field:cavy, hasBoolean=true, hasPhrase=false)");
     splitOnWhitespace = oldSplitOnWhitespace;
   }
    


[25/50] lucene-solr:jira/solr-8593: LUCENE-7595: Disable another test not compatible with RamUsageTester

Posted by kr...@apache.org.
LUCENE-7595: Disable another test not compatible with RamUsageTester


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

Branch: refs/heads/jira/solr-8593
Commit: d65c02e8cc14f03389c2426ea3d3ddd75e12b1ec
Parents: db9190d
Author: Uwe Schindler <us...@apache.org>
Authored: Thu Dec 29 22:56:54 2016 +0100
Committer: Uwe Schindler <us...@apache.org>
Committed: Thu Dec 29 22:56:54 2016 +0100

----------------------------------------------------------------------
 .../core/src/test/org/apache/lucene/search/TestLRUQueryCache.java  | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d65c02e8/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
index 9ebacf7..3acc3ea 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
@@ -381,6 +381,8 @@ public class TestLRUQueryCache extends LuceneTestCase {
   // by the cache itself, not cache entries, and we want to make sure that
   // memory usage is not grossly underestimated.
   public void testRamBytesUsedConstantEntryOverhead() throws IOException {
+    assumeFalse("LUCENE-7595: RamUsageTester does not work exact in Java 9 (estimations for maps and lists)", Constants.JRE_IS_MINIMUM_JAVA9);
+    
     final LRUQueryCache queryCache = new LRUQueryCache(1000000, 10000000, context -> true);
 
     final RamUsageTester.Accumulator acc = new RamUsageTester.Accumulator() {


[35/50] lucene-solr:jira/solr-8593: SOLR-9880 Fix concurrency bugs in tests.

Posted by kr...@apache.org.
SOLR-9880 Fix concurrency bugs in tests.


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

Branch: refs/heads/jira/solr-8593
Commit: fb2800b1497a67493c7f8944bda22f590bb9cc6b
Parents: 832d02b
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Sat Dec 31 20:46:32 2016 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Sat Dec 31 20:46:32 2016 +0100

----------------------------------------------------------------------
 .../apache/solr/metrics/reporters/SolrGangliaReporterTest.java    | 3 ++-
 .../apache/solr/metrics/reporters/SolrGraphiteReporterTest.java   | 3 ++-
 2 files changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fb2800b1/solr/core/src/test/org/apache/solr/metrics/reporters/SolrGangliaReporterTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/reporters/SolrGangliaReporterTest.java b/solr/core/src/test/org/apache/solr/metrics/reporters/SolrGangliaReporterTest.java
index b5b0f85..d9675ef 100644
--- a/solr/core/src/test/org/apache/solr/metrics/reporters/SolrGangliaReporterTest.java
+++ b/solr/core/src/test/org/apache/solr/metrics/reporters/SolrGangliaReporterTest.java
@@ -73,7 +73,8 @@ public class SolrGangliaReporterTest extends SolrTestCaseJ4 {
     gangliaReporter.start();
     Thread.sleep(5000);
     assertTrue(names.size() >= 3);
-    for (String name : names) {
+    String[] frozenNames = (String[])names.toArray(new String[names.size()]);
+    for (String name : frozenNames) {
       assertTrue(name, name.startsWith("test.solr.node.cores."));
     }
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fb2800b1/solr/core/src/test/org/apache/solr/metrics/reporters/SolrGraphiteReporterTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/reporters/SolrGraphiteReporterTest.java b/solr/core/src/test/org/apache/solr/metrics/reporters/SolrGraphiteReporterTest.java
index 6773e0c..3d1c482 100644
--- a/solr/core/src/test/org/apache/solr/metrics/reporters/SolrGraphiteReporterTest.java
+++ b/solr/core/src/test/org/apache/solr/metrics/reporters/SolrGraphiteReporterTest.java
@@ -69,7 +69,8 @@ public class SolrGraphiteReporterTest extends SolrTestCaseJ4 {
       assertTrue(reporter instanceof SolrGraphiteReporter);
       Thread.sleep(5000);
       assertTrue(mock.lines.size() >= 3);
-      for (String line : mock.lines) {
+      String[] frozenLines = (String[])mock.lines.toArray(new String[mock.lines.size()]);
+      for (String line : frozenLines) {
         assertTrue(line, line.startsWith("test.solr.node.cores."));
       }
     } finally {


[21/50] lucene-solr:jira/solr-8593: LUCENE-5325: Add test for missing values in sorts

Posted by kr...@apache.org.
LUCENE-5325: Add test for missing values in sorts


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

Branch: refs/heads/jira/solr-8593
Commit: a4335c0e9f01275c7d6e807813d9818b6e59d76e
Parents: 0072382
Author: Alan Woodward <ro...@apache.org>
Authored: Thu Dec 29 14:44:11 2016 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Thu Dec 29 20:26:36 2016 +0000

----------------------------------------------------------------------
 .../org/apache/lucene/search/TestDoubleValuesSource.java    | 9 +++++++++
 .../test/org/apache/lucene/search/TestLongValuesSource.java | 9 +++++++++
 2 files changed, 18 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a4335c0e/lucene/core/src/test/org/apache/lucene/search/TestDoubleValuesSource.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestDoubleValuesSource.java b/lucene/core/src/test/org/apache/lucene/search/TestDoubleValuesSource.java
index 3860963..13a5168 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestDoubleValuesSource.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestDoubleValuesSource.java
@@ -53,6 +53,8 @@ public class TestDoubleValuesSource extends LuceneTestCase {
       document.add(new NumericDocValuesField("long", random().nextLong()));
       document.add(new FloatDocValuesField("float", random().nextFloat()));
       document.add(new DoubleDocValuesField("double", random().nextDouble()));
+      if (i == 545)
+        document.add(new DoubleDocValuesField("onefield", 45.72));
       iw.addDocument(document);
     }
     reader = iw.getReader();
@@ -67,6 +69,13 @@ public class TestDoubleValuesSource extends LuceneTestCase {
     super.tearDown();
   }
 
+  public void testSortMissing() throws Exception {
+    DoubleValuesSource onefield = DoubleValuesSource.fromDoubleField("onefield");
+    TopDocs results = searcher.search(new MatchAllDocsQuery(), 1, new Sort(onefield.getSortField(true)));
+    FieldDoc first = (FieldDoc) results.scoreDocs[0];
+    assertEquals(45.72, first.fields[0]);
+  }
+
   public void testSimpleFieldEquivalences() throws Exception {
     checkSorts(new MatchAllDocsQuery(), new Sort(new SortField("int", SortField.Type.INT, random().nextBoolean())));
     checkSorts(new MatchAllDocsQuery(), new Sort(new SortField("long", SortField.Type.LONG, random().nextBoolean())));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a4335c0e/lucene/core/src/test/org/apache/lucene/search/TestLongValuesSource.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestLongValuesSource.java b/lucene/core/src/test/org/apache/lucene/search/TestLongValuesSource.java
index 9148ad5..8b20be5 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestLongValuesSource.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestLongValuesSource.java
@@ -49,6 +49,8 @@ public class TestLongValuesSource extends LuceneTestCase {
       document.add(newTextField("oddeven", (i % 2 == 0) ? "even" : "odd", Field.Store.NO));
       document.add(new NumericDocValuesField("int", random().nextInt()));
       document.add(new NumericDocValuesField("long", random().nextLong()));
+      if (i == 545)
+        document.add(new NumericDocValuesField("onefield", 45));
       iw.addDocument(document);
     }
     reader = iw.getReader();
@@ -63,6 +65,13 @@ public class TestLongValuesSource extends LuceneTestCase {
     super.tearDown();
   }
 
+  public void testSortMissing() throws Exception {
+    LongValuesSource onefield = LongValuesSource.fromLongField("onefield");
+    TopDocs results = searcher.search(new MatchAllDocsQuery(), 1, new Sort(onefield.getSortField(true)));
+    FieldDoc first = (FieldDoc) results.scoreDocs[0];
+    assertEquals(45L, first.fields[0]);
+  }
+
   public void testSimpleFieldEquivalences() throws Exception {
     checkSorts(new MatchAllDocsQuery(), new Sort(new SortField("int", SortField.Type.INT, random().nextBoolean())));
     checkSorts(new MatchAllDocsQuery(), new Sort(new SortField("long", SortField.Type.LONG, random().nextBoolean())));


[34/50] lucene-solr:jira/solr-8593: SOLR-9495: Update CHANGES.txt

Posted by kr...@apache.org.
SOLR-9495: Update CHANGES.txt


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

Branch: refs/heads/jira/solr-8593
Commit: 832d02bf494c8fea02398db31b55de4314f2be8a
Parents: 6167618
Author: Joel Bernstein <jb...@apache.org>
Authored: Fri Dec 30 20:39:57 2016 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Fri Dec 30 20:39:57 2016 -0500

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


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/832d02bf/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 6a99617..02167f3 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -306,6 +306,8 @@ Bug Fixes
 
 * SOLR-9900: fix false positives on range queries with ReversedWildcardFilterFactory (Yonik Seeley via Mikhail Khludnev)
 
+* SOLR-9495: AIOBE with confusing message for incomplete sort spec in Streaming Expression (Gus Heck, Joel Bernstein)
+
 Other Changes
 ----------------------
 


[29/50] lucene-solr:jira/solr-8593: LUCENE-7606: Normalization with CustomAnalyzer would only apply the last token filter.

Posted by kr...@apache.org.
LUCENE-7606: Normalization with CustomAnalyzer would only apply the last token filter.


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

Branch: refs/heads/jira/solr-8593
Commit: 26ee8e9bea70e857aa61764020337ce675066bd1
Parents: 3ccd15a
Author: Adrien Grand <jp...@gmail.com>
Authored: Fri Dec 30 10:30:40 2016 +0100
Committer: Adrien Grand <jp...@gmail.com>
Committed: Fri Dec 30 10:30:40 2016 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  3 +++
 .../lucene/analysis/custom/CustomAnalyzer.java  |  2 +-
 .../analysis/custom/TestCustomAnalyzer.java     | 21 ++++++++++++++++++++
 .../apache/lucene/analysis/custom/mapping1.txt  |  1 +
 .../apache/lucene/analysis/custom/mapping2.txt  |  1 +
 5 files changed, 27 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/26ee8e9b/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 12b615d..4c49560 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -129,6 +129,9 @@ Bug Fixes
   using helpers for exclusive bounds that are consistent with Double.compare.
   (Adrien Grand, Dawid Weiss)
 
+* LUCENE-7606: Normalization with CustomAnalyzer would only apply the last
+  token filter. (Adrien Grand)
+
 Improvements
 
 * LUCENE-6824: TermAutomatonQuery now rewrites to TermQuery,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/26ee8e9b/lucene/analysis/common/src/java/org/apache/lucene/analysis/custom/CustomAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/custom/CustomAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/custom/CustomAnalyzer.java
index 466642c..1cfdfe3 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/custom/CustomAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/custom/CustomAnalyzer.java
@@ -145,7 +145,7 @@ public final class CustomAnalyzer extends Analyzer {
     for (TokenFilterFactory filter : tokenFilters) {
       if (filter instanceof MultiTermAwareComponent) {
         filter = (TokenFilterFactory) ((MultiTermAwareComponent) filter).getMultiTermComponent();
-        result = filter.create(in);
+        result = filter.create(result);
       }
     }
     return result;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/26ee8e9b/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/TestCustomAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/TestCustomAnalyzer.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/TestCustomAnalyzer.java
index aa69b70..d929bfd 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/TestCustomAnalyzer.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/TestCustomAnalyzer.java
@@ -30,6 +30,7 @@ import org.apache.lucene.analysis.CharFilter;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.analysis.charfilter.HTMLStripCharFilterFactory;
+import org.apache.lucene.analysis.charfilter.MappingCharFilterFactory;
 import org.apache.lucene.analysis.core.KeywordTokenizerFactory;
 import org.apache.lucene.analysis.core.LowerCaseFilterFactory;
 import org.apache.lucene.analysis.core.LowerCaseTokenizer;
@@ -479,4 +480,24 @@ public class TestCustomAnalyzer extends BaseTokenStreamTestCase {
     assertEquals(new BytesRef("2A"), analyzer2.normalize("dummy", "0�"));
   }
 
+  public void testNormalizationWithMultipleTokenFilters() throws IOException {
+    CustomAnalyzer analyzer = CustomAnalyzer.builder()
+        // none of these components are multi-term aware so they should not be applied
+        .withTokenizer(WhitespaceTokenizerFactory.class, Collections.emptyMap())
+        .addTokenFilter(LowerCaseFilterFactory.class, Collections.emptyMap())
+        .addTokenFilter(ASCIIFoldingFilterFactory.class, Collections.emptyMap())
+        .build();
+    assertEquals(new BytesRef("a b e"), analyzer.normalize("dummy", "� B �"));
+  }
+
+  public void testNormalizationWithMultiplCharFilters() throws IOException {
+    CustomAnalyzer analyzer = CustomAnalyzer.builder()
+        // none of these components are multi-term aware so they should not be applied
+        .withTokenizer(WhitespaceTokenizerFactory.class, Collections.emptyMap())
+        .addCharFilter(MappingCharFilterFactory.class, new HashMap<>(Collections.singletonMap("mapping", "org/apache/lucene/analysis/custom/mapping1.txt")))
+        .addCharFilter(MappingCharFilterFactory.class, new HashMap<>(Collections.singletonMap("mapping", "org/apache/lucene/analysis/custom/mapping2.txt")))
+        .build();
+    assertEquals(new BytesRef("e f c"), analyzer.normalize("dummy", "a b c"));
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/26ee8e9b/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/mapping1.txt
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/mapping1.txt b/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/mapping1.txt
new file mode 100644
index 0000000..40aaf5a
--- /dev/null
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/mapping1.txt
@@ -0,0 +1 @@
+"a" => "e"

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/26ee8e9b/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/mapping2.txt
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/mapping2.txt b/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/mapping2.txt
new file mode 100644
index 0000000..cac0bea
--- /dev/null
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/custom/mapping2.txt
@@ -0,0 +1 @@
+"b" => "f"


[09/50] lucene-solr:jira/solr-8593: tests: bump up fudge

Posted by kr...@apache.org.
tests: bump up fudge


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

Branch: refs/heads/jira/solr-8593
Commit: a5e5c4a04385eb030aac1ec6126ff9b82407158f
Parents: 2781145
Author: markrmiller <ma...@apache.org>
Authored: Thu Dec 29 05:40:45 2016 -0500
Committer: markrmiller <ma...@apache.org>
Committed: Thu Dec 29 05:40:45 2016 -0500

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


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a5e5c4a0/solr/core/src/test/org/apache/solr/update/SoftAutoCommitTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/SoftAutoCommitTest.java b/solr/core/src/test/org/apache/solr/update/SoftAutoCommitTest.java
index 4106884..c9c9691 100644
--- a/solr/core/src/test/org/apache/solr/update/SoftAutoCommitTest.java
+++ b/solr/core/src/test/org/apache/solr/update/SoftAutoCommitTest.java
@@ -107,7 +107,7 @@ public class SoftAutoCommitTest extends AbstractSolrTestCase {
     monitor.assertSaneOffers();
 
     // Wait for the soft commit with some fudge
-    Long soft529 = monitor.soft.poll(softCommitWaitMillis * 2, MILLISECONDS);
+    Long soft529 = monitor.soft.poll(softCommitWaitMillis * 3, MILLISECONDS);
     assertNotNull("soft529 wasn't fast enough", soft529);
     monitor.assertSaneOffers();
 


[20/50] lucene-solr:jira/solr-8593: SOLR-9905: Update CHANGES.txt

Posted by kr...@apache.org.
SOLR-9905: Update CHANGES.txt


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

Branch: refs/heads/jira/solr-8593
Commit: 00723827ff5ad5c129d3d8487d2c64469ea03239
Parents: 7dcb557
Author: Joel Bernstein <jb...@apache.org>
Authored: Thu Dec 29 14:42:31 2016 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Thu Dec 29 14:42:31 2016 -0500

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


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/00723827/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 1383859..b509e23 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -202,6 +202,8 @@ New Features
 * SOLR-9897: Add hl.requireFieldMatch toggle support when using the UnifiedHighlighter.  Defaults to false like the
   other highlighters that support this. (David Smiley)
 
+* SOLR-9905: Add NullStream to isolate the performance of the ExportWriter (Joel Bernstein)
+
 Optimizations
 ----------------------
 * SOLR-9704: Facet Module / JSON Facet API: Optimize blockChildren facets that have


[02/50] lucene-solr:jira/solr-8593: LUCENE-7605: Use codec-specific impl of live docs when sorting.

Posted by kr...@apache.org.
LUCENE-7605: Use codec-specific impl of live docs when sorting.


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

Branch: refs/heads/jira/solr-8593
Commit: dc6dcdda8078eb9f100fd6c66b5d488d057b019b
Parents: e4ef423
Author: Adrien Grand <jp...@gmail.com>
Authored: Wed Dec 28 20:12:02 2016 +0100
Committer: Adrien Grand <jp...@gmail.com>
Committed: Wed Dec 28 20:12:02 2016 +0100

----------------------------------------------------------------------
 .../lucene/index/DocumentsWriterPerThread.java      | 16 ++++++++--------
 1 file changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dc6dcdda/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
index 49d03ad..48901e5 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
@@ -33,10 +33,10 @@ import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FlushInfo;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.TrackingDirectoryWrapper;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.ByteBlockPool.Allocator;
 import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
 import org.apache.lucene.util.Counter;
-import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.InfoStream;
 import org.apache.lucene.util.IntBlockPool;
 import org.apache.lucene.util.MutableBits;
@@ -496,15 +496,15 @@ class DocumentsWriterPerThread {
     return filesToDelete;
   }
 
-  private MutableBits sortLiveDocs(MutableBits bits, Sorter.DocMap sortMap) {
-    assert bits != null && sortMap != null;
-    FixedBitSet bitSet = new FixedBitSet(bits.length());
-    for (int i = 0; i < bits.length(); i++) {
-      if (bits.get(i)) {
-        bitSet.set(sortMap.oldToNew(i));
+  private MutableBits sortLiveDocs(Bits liveDocs, Sorter.DocMap sortMap) throws IOException {
+    assert liveDocs != null && sortMap != null;
+    MutableBits sortedLiveDocs = codec.liveDocsFormat().newLiveDocs(liveDocs.length());
+    for (int i = 0; i < liveDocs.length(); i++) {
+      if (liveDocs.get(i) == false) {
+        sortedLiveDocs.clear(sortMap.oldToNew(i));
       }
     }
-    return bitSet;
+    return sortedLiveDocs;
   }
 
   /**


[31/50] lucene-solr:jira/solr-8593: SOLR-9684: Add schedule Streaming Expression

Posted by kr...@apache.org.
SOLR-9684: Add schedule Streaming Expression


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

Branch: refs/heads/jira/solr-8593
Commit: f3fe487970f1e21300bd556d226461a2d51b00f9
Parents: cc862d8
Author: Joel Bernstein <jb...@apache.org>
Authored: Fri Dec 30 14:34:00 2016 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Fri Dec 30 14:34:16 2016 -0500

----------------------------------------------------------------------
 .../org/apache/solr/handler/StreamHandler.java  |   1 +
 .../client/solrj/io/stream/SchedulerStream.java | 161 +++++++++++++++++++
 .../solrj/io/stream/StreamExpressionTest.java   | 149 +++++++++++++++++
 3 files changed, 311 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f3fe4879/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 13ce636..1610fea 100644
--- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
@@ -140,6 +140,7 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
       .withFunctionName("fetch", FetchStream.class)
       .withFunctionName("executor", ExecutorStream.class)
       .withFunctionName("null", NullStream.class)
+      .withFunctionName("schedule", SchedulerStream.class)
       // metrics
       .withFunctionName("min", MinMetric.class)
       .withFunctionName("max", MaxMetric.class)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f3fe4879/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SchedulerStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SchedulerStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SchedulerStream.java
new file mode 100644
index 0000000..f8506b9
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SchedulerStream.java
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.client.solrj.io.stream;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+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.comp.StreamComparator;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType;
+import org.apache.solr.client.solrj.io.stream.expr.Expressible;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExplanation;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The scheduler wraps two topics that represent high priority and low priority task queues.
+ * Each time the scheduler is called it will check to see if there are any high priority tasks in the queue. If there
+ * are high priority tasks, then the high priority queue will be read until it returns the EOF Tuple.
+ *
+ * If there are no tasks in the high priority queue, then the lower priority task queue will be opened and read until the EOF Tuple is
+ * returned.
+ *
+ * The scheduler is designed to be wrapped by the executor function and a daemon function can be used to call the executor iteratively.
+ **/
+
+public class SchedulerStream extends TupleStream implements Expressible {
+
+  private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private PushBackStream highPriorityTasks;
+  private PushBackStream tasks;
+  private TupleStream currentStream;
+
+  public SchedulerStream(StreamExpression expression, StreamFactory factory) throws IOException {
+    // grab all parameters out
+    List<StreamExpression> streamExpressions = factory.getExpressionOperandsRepresentingTypes(expression, Expressible.class, TupleStream.class);
+
+
+    if(2 != streamExpressions.size()){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting a single stream but found %d",expression, streamExpressions.size()));
+    }
+
+    TupleStream stream1 = factory.constructStream(streamExpressions.get(0));
+    TupleStream stream2 = factory.constructStream(streamExpressions.get(1));
+
+    if(!(stream1 instanceof TopicStream) || !(stream2 instanceof TopicStream)) {
+      throw new IOException("The scheduler expects both stream parameters to be topics.");
+    }
+
+    init(new PushBackStream(stream1), new PushBackStream(stream2));
+  }
+
+  private void init(PushBackStream stream1, PushBackStream stream2) throws IOException{
+    this.highPriorityTasks = stream1;
+    this.tasks = stream2;
+  }
+
+  @Override
+  public StreamExpression toExpression(StreamFactory factory) throws IOException {
+    return toExpression(factory, true);
+  }
+
+  private StreamExpression toExpression(StreamFactory factory, boolean includeStreams) throws IOException {
+
+    // function name
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
+
+    // stream
+    if(includeStreams) {
+      if (highPriorityTasks instanceof Expressible) {
+        expression.addParameter(((Expressible) highPriorityTasks).toExpression(factory));
+      } else {
+        throw new IOException("The SchedulerStream contains a non-expressible TupleStream - it cannot be converted to an expression");
+      }
+
+      if (tasks instanceof Expressible) {
+        expression.addParameter(((Expressible) tasks).toExpression(factory));
+      } else {
+        throw new IOException("The SchedulerStream contains a non-expressible TupleStream - it cannot be converted to an expression");
+      }
+    }
+
+    return expression;
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+
+    return new StreamExplanation(getStreamNodeId().toString())
+        .withChildren(new Explanation[]{
+            highPriorityTasks.toExplanation(factory), tasks.toExplanation(factory)
+        })
+        .withFunctionName(factory.getFunctionName(this.getClass()))
+        .withImplementingClass(this.getClass().getName())
+        .withExpressionType(ExpressionType.STREAM_DECORATOR)
+        .withExpression(toExpression(factory, false).toString());
+  }
+
+  public void setStreamContext(StreamContext streamContext) {
+    this.highPriorityTasks.setStreamContext(streamContext);
+    tasks.setStreamContext(streamContext);
+  }
+
+  public List<TupleStream> children() {
+    List<TupleStream> l =  new ArrayList();
+    l.add(highPriorityTasks);
+    l.add(tasks);
+    return l;
+  }
+
+  public void open() throws IOException {
+    highPriorityTasks.open();
+    Tuple tuple = highPriorityTasks.read();
+    if(tuple.EOF) {
+      highPriorityTasks.close();
+      tasks.open();
+      currentStream = tasks;
+    } else {
+      highPriorityTasks.pushBack(tuple);
+      currentStream = highPriorityTasks;
+    }
+  }
+
+  public void close() throws IOException {
+      currentStream.close();
+  }
+
+  public Tuple read() throws IOException {
+    return currentStream.read();
+  }
+
+  public StreamComparator getStreamSort(){
+    return null;
+  }
+
+  public int getCost() {
+    return 0;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f3fe4879/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 0c9d5b3..3bfe129 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
@@ -2816,6 +2816,155 @@ public class StreamExpressionTest extends SolrCloudTestCase {
   }
 
   @Test
+  public void testSchedulerStream() throws Exception {
+    Assume.assumeTrue(!useAlias);
+
+    new UpdateRequest()
+        .add(id, "0", "a_s", "hello1", "a_i", "0", "a_f", "1")
+        .add(id, "2", "a_s", "hello1", "a_i", "2", "a_f", "2")
+        .add(id, "3", "a_s", "hello1", "a_i", "3", "a_f", "3")
+        .add(id, "4", "a_s", "hello1", "a_i", "4", "a_f", "4")
+        .add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "5")
+        .add(id, "5", "a_s", "hello", "a_i", "10", "a_f", "6")
+        .add(id, "6", "a_s", "hello", "a_i", "11", "a_f", "7")
+        .add(id, "7", "a_s", "hello", "a_i", "12", "a_f", "8")
+        .add(id, "8", "a_s", "hello", "a_i", "13", "a_f", "9")
+        .add(id, "9", "a_s", "hello1", "a_i", "14", "a_f", "10")
+        .commit(cluster.getSolrClient(), COLLECTIONORALIAS);
+
+    StreamFactory factory = new StreamFactory()
+        .withCollectionZkHost("collection1", cluster.getZkServer().getZkAddress())
+        .withFunctionName("topic", TopicStream.class)
+        .withFunctionName("schedule", SchedulerStream.class);
+
+    StreamExpression expression;
+    TupleStream stream;
+    List<Tuple> tuples;
+
+    SolrClientCache cache = new SolrClientCache();
+
+    try {
+      FieldComparator comp = new FieldComparator("a_i", ComparatorOrder.ASCENDING);
+
+      expression = StreamExpressionParser.parse("schedule(topic(collection1, collection1, q=\"a_s:hello\", fl=\"id,a_i\", id=1000000, initialCheckpoint=0)," +
+          "topic(collection1, collection1, q=\"a_s:hello1\", fl=\"id,a_i\", id=2000000, initialCheckpoint=0))");
+      stream = factory.constructStream(expression);
+      StreamContext context = new StreamContext();
+      context.setSolrClientCache(cache);
+      stream.setStreamContext(context);
+      tuples = getTuples(stream);
+
+      Collections.sort(tuples, comp);
+      //The tuples from the first topic (high priority) should be returned.
+
+      assertEquals(tuples.size(), 4);
+      assertOrder(tuples, 5, 6, 7, 8);
+
+      expression = StreamExpressionParser.parse("schedule(topic(collection1, collection1, q=\"a_s:hello\", fl=\"id,a_i\", id=1000000, initialCheckpoint=0)," +
+          "topic(collection1, collection1, q=\"a_s:hello1\", fl=\"id,a_i\", id=2000000, initialCheckpoint=0))");
+      stream = factory.constructStream(expression);
+      context = new StreamContext();
+      context.setSolrClientCache(cache);
+      stream.setStreamContext(context);
+      tuples = getTuples(stream);
+      Collections.sort(tuples, comp);
+
+      //The Tuples from the second topic (Low priority) should be returned.
+      assertEquals(tuples.size(), 6);
+      assertOrder(tuples, 0, 1, 2, 3, 4, 9);
+
+      expression = StreamExpressionParser.parse("schedule(topic(collection1, collection1, q=\"a_s:hello\", fl=\"id,a_i\", id=1000000, initialCheckpoint=0)," +
+          "topic(collection1, collection1, q=\"a_s:hello1\", fl=\"id,a_i\", id=2000000, initialCheckpoint=0))");
+      stream = factory.constructStream(expression);
+      context = new StreamContext();
+      context.setSolrClientCache(cache);
+      stream.setStreamContext(context);
+      tuples = getTuples(stream);
+
+      //Both queus are empty.
+      assertEquals(tuples.size(), 0);
+
+    } finally {
+      cache.close();
+    }
+  }
+
+  @Test
+  public void testParallelSchedulerStream() throws Exception {
+    Assume.assumeTrue(!useAlias);
+
+    new UpdateRequest()
+        .add(id, "0", "a_s", "hello1", "a_i", "0", "a_f", "1")
+        .add(id, "2", "a_s", "hello1", "a_i", "2", "a_f", "2")
+        .add(id, "3", "a_s", "hello1", "a_i", "3", "a_f", "3")
+        .add(id, "4", "a_s", "hello1", "a_i", "4", "a_f", "4")
+        .add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "5")
+        .add(id, "5", "a_s", "hello", "a_i", "10", "a_f", "6")
+        .add(id, "6", "a_s", "hello", "a_i", "11", "a_f", "7")
+        .add(id, "7", "a_s", "hello", "a_i", "12", "a_f", "8")
+        .add(id, "8", "a_s", "hello", "a_i", "13", "a_f", "9")
+        .add(id, "9", "a_s", "hello1", "a_i", "14", "a_f", "10")
+        .commit(cluster.getSolrClient(), COLLECTIONORALIAS);
+
+    StreamFactory factory = new StreamFactory()
+        .withCollectionZkHost("collection1", cluster.getZkServer().getZkAddress())
+        .withFunctionName("topic", TopicStream.class)
+        .withFunctionName("parallel", ParallelStream.class)
+        .withFunctionName("schedule", SchedulerStream.class);
+
+    StreamExpression expression;
+    TupleStream stream;
+    List<Tuple> tuples;
+
+    SolrClientCache cache = new SolrClientCache();
+
+    try {
+      FieldComparator comp = new FieldComparator("a_i", ComparatorOrder.ASCENDING);
+
+      expression = StreamExpressionParser.parse("parallel(collection1, workers=2, sort=\"_version_ asc\", schedule(topic(collection1, collection1, q=\"a_s:hello\", fl=\"id,a_i\", id=1000000, initialCheckpoint=0, partitionKeys=id)," +
+          "topic(collection1, collection1, q=\"a_s:hello1\", fl=\"id,a_i\", id=2000000, initialCheckpoint=0, partitionKeys=id)))");
+      stream = factory.constructStream(expression);
+      StreamContext context = new StreamContext();
+      context.setSolrClientCache(cache);
+      stream.setStreamContext(context);
+      tuples = getTuples(stream);
+
+      Collections.sort(tuples, comp);
+      //The tuples from the first topic (high priority) should be returned.
+
+      assertEquals(tuples.size(), 4);
+      assertOrder(tuples, 5, 6, 7, 8);
+
+      expression = StreamExpressionParser.parse("parallel(collection1, workers=2, sort=\"_version_ asc\", schedule(topic(collection1, collection1, q=\"a_s:hello\", fl=\"id,a_i\", id=1000000, initialCheckpoint=0, partitionKeys=id)," +
+          "topic(collection1, collection1, q=\"a_s:hello1\", fl=\"id,a_i\", id=2000000, initialCheckpoint=0, partitionKeys=id)))");
+      stream = factory.constructStream(expression);
+      context = new StreamContext();
+      context.setSolrClientCache(cache);
+      stream.setStreamContext(context);
+      tuples = getTuples(stream);
+      Collections.sort(tuples, comp);
+
+      //The Tuples from the second topic (Low priority) should be returned.
+      assertEquals(tuples.size(), 6);
+      assertOrder(tuples, 0, 1, 2, 3, 4, 9);
+
+      expression = StreamExpressionParser.parse("parallel(collection1, workers=2, sort=\"_version_ asc\", schedule(topic(collection1, collection1, q=\"a_s:hello\", fl=\"id,a_i\", id=1000000, initialCheckpoint=0, partitionKeys=id)," +
+          "topic(collection1, collection1, q=\"a_s:hello1\", fl=\"id,a_i\", id=2000000, initialCheckpoint=0, partitionKeys=id)))");
+      stream = factory.constructStream(expression);
+      context = new StreamContext();
+      context.setSolrClientCache(cache);
+      stream.setStreamContext(context);
+      tuples = getTuples(stream);
+
+      //Both queus are empty.
+      assertEquals(tuples.size(), 0);
+
+    } finally {
+      cache.close();
+    }
+  }
+
+  @Test
   public void testParallelTopicStream() throws Exception {
 
     Assume.assumeTrue(!useAlias);


[18/50] lucene-solr:jira/solr-8593: LUCENE-7595: Fix bug with RamUsageTester incorrectly handling Iterables outside Java Runtime

Posted by kr...@apache.org.
LUCENE-7595: Fix bug with RamUsageTester incorrectly handling Iterables outside Java Runtime


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

Branch: refs/heads/jira/solr-8593
Commit: db9190db9372ae88a7392a7186397441ce070a96
Parents: 3f24fd8
Author: Uwe Schindler <us...@apache.org>
Authored: Thu Dec 29 20:31:47 2016 +0100
Committer: Uwe Schindler <us...@apache.org>
Committed: Thu Dec 29 20:31:47 2016 +0100

----------------------------------------------------------------------
 .../src/java/org/apache/lucene/util/RamUsageTester.java            | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/db9190db/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java b/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java
index daf81a9..6437d8e 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java
@@ -152,7 +152,7 @@ public final class RamUsageTester {
           }
           
           boolean needsReflection = true;
-          if (Constants.JRE_IS_MINIMUM_JAVA9) {
+          if (Constants.JRE_IS_MINIMUM_JAVA9 && obClazz.getName().startsWith("java.")) {
             // Java 9: Best guess for some known types, as we cannot precisely look into runtime classes:
             final ToLongFunction<Object> func = SIMPLE_TYPES.get(obClazz);
             if (func != null) { // some simple type like String where the size is easy to get from public properties


[17/50] lucene-solr:jira/solr-8593: SOLR-9132: Cut over some more tests

Posted by kr...@apache.org.
SOLR-9132: Cut over some more tests


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

Branch: refs/heads/jira/solr-8593
Commit: 12aff1cfcc48d7c89008447d482bf610242e0431
Parents: fa959ad
Author: Alan Woodward <ro...@apache.org>
Authored: Thu Oct 27 16:50:28 2016 +0100
Committer: Alan Woodward <ro...@apache.org>
Committed: Thu Dec 29 12:22:40 2016 +0000

----------------------------------------------------------------------
 .../java/org/apache/solr/cloud/Overseer.java    |   2 +-
 .../solr/cloud/CollectionsAPISolrJTest.java     | 470 +++++++------------
 .../DeleteLastCustomShardedReplicaTest.java     | 104 +---
 .../org/apache/solr/cloud/DeleteShardTest.java  | 211 +++------
 .../cloud/OverseerModifyCollectionTest.java     |  92 ++--
 .../apache/solr/cloud/OverseerRolesTest.java    | 165 +++----
 .../apache/solr/cloud/OverseerStatusTest.java   |  55 +--
 .../apache/solr/cloud/RemoteQueryErrorTest.java |  53 +--
 .../solr/cloud/TestDownShardTolerantSearch.java |  40 +-
 .../TestExclusionRuleCollectionAccess.java      |  38 +-
 .../PKIAuthenticationIntegrationTest.java       |  40 +-
 .../solrj/request/CollectionAdminRequest.java   |   2 +
 12 files changed, 457 insertions(+), 815 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12aff1cf/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 d7285fa..a618874 100644
--- a/solr/core/src/java/org/apache/solr/cloud/Overseer.java
+++ b/solr/core/src/java/org/apache/solr/cloud/Overseer.java
@@ -371,7 +371,7 @@ public class Overseer implements Closeable {
             return Collections.singletonList(new SliceMutator(getZkStateReader()).updateShardState(clusterState, message));
           case QUIT:
             if (myId.equals(message.get("id"))) {
-              log.info("Quit command received {}", LeaderElector.getNodeName(myId));
+              log.info("Quit command received {} {}", message, LeaderElector.getNodeName(myId));
               overseerCollectionConfigSetProcessor.close();
               close();
             } else {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12aff1cf/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
index b04bfbc..616b657 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
@@ -16,70 +16,47 @@
  */
 package org.apache.solr.cloud;
 
-import java.io.File;
 import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
-import java.util.Objects;
-import java.util.Properties;
-import java.util.concurrent.TimeUnit;
 
-import org.apache.commons.codec.binary.StringUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.client.solrj.request.CoreAdminRequest;
+import org.apache.solr.client.solrj.request.CoreStatus;
 import org.apache.solr.client.solrj.response.CollectionAdminResponse;
-import org.apache.solr.client.solrj.response.CoreAdminResponse;
-import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.ClusterProperties;
 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.ZkStateReader;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.util.NamedList;
-import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
-import static org.apache.solr.cloud.ReplicaPropertiesBase.verifyUniqueAcrossCollection;
-
 @LuceneTestCase.Slow
-public class CollectionsAPISolrJTest extends AbstractFullDistribZkTestBase {
+public class CollectionsAPISolrJTest extends SolrCloudTestCase {
 
-  @Test
-  public void test() throws Exception {
-    testCreateAndDeleteCollection();
-    testCreateAndDeleteShard();
-    testReloadCollection();
-    testCreateAndDeleteAlias();
-    testSplitShard();
-    testCreateCollectionWithPropertyParam();
-    testAddAndDeleteReplica();
-    testClusterProp();
-    testAddAndRemoveRole();
-    testOverseerStatus();
-    testList();
-    testAddAndDeleteReplicaProp();
-    testBalanceShardUnique();
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(4)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
   }
 
-  protected void testCreateAndDeleteCollection() throws Exception {
+  @Test
+  public void testCreateAndDeleteCollection() throws Exception {
     String collectionName = "solrj_test";
-    CollectionAdminRequest.Create createCollectionRequest = new CollectionAdminRequest.Create()
-            .setCollectionName(collectionName)
-            .setNumShards(2)
-            .setReplicationFactor(2)
-            .setConfigName("conf1")
-            .setRouterField("myOwnField")
-            .setStateFormat(1);
-
-    CollectionAdminResponse response = createCollectionRequest.process(cloudClient);
+    CollectionAdminResponse response = CollectionAdminRequest.createCollection(collectionName, "conf", 2, 2)
+        .setStateFormat(1)
+        .process(cluster.getSolrClient());
 
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
@@ -91,57 +68,44 @@ public class CollectionsAPISolrJTest extends AbstractFullDistribZkTestBase {
       assertTrue(status.get("QTime") > 0);
     }
 
-    cloudClient.setDefaultCollection(collectionName);
-    CollectionAdminRequest.Delete deleteCollectionRequest = new CollectionAdminRequest.Delete()
-            .setCollectionName(collectionName);
-    response = deleteCollectionRequest.process(cloudClient);
+    response = CollectionAdminRequest.deleteCollection(collectionName).process(cluster.getSolrClient());
 
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
     Map<String,NamedList<Integer>> nodesStatus = response.getCollectionNodesStatus();
-    assertNull("Deleted collection " + collectionName + "still exists",
-        cloudClient.getZkStateReader().getClusterState().getCollectionOrNull(collectionName));
     assertEquals(4, nodesStatus.size());
-    
+
+    waitForState("Expected " + collectionName + " to disappear from cluster state", collectionName, (n, c) -> c == null);
+
     // Test Creating a collection with new stateformat.
     collectionName = "solrj_newstateformat";
-    createCollectionRequest = new CollectionAdminRequest.Create()
-            .setCollectionName(collectionName)
-            .setNumShards(2)
-            .setConfigName("conf1")
-            .setStateFormat(2);
 
-    response = createCollectionRequest.process(cloudClient);
+    response = CollectionAdminRequest.createCollection(collectionName, "conf", 2, 2)
+        .setStateFormat(2)
+        .process(cluster.getSolrClient());
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
 
-    waitForRecoveriesToFinish(collectionName, false);
-    assertTrue("Collection state does not exist",
-        cloudClient.getZkStateReader().getZkClient()
-            .exists(ZkStateReader.getCollectionPath(collectionName), true));
+    waitForState("Expected " + collectionName + " to appear in cluster state", collectionName, (n, c) -> c != null);
 
   }
-  
-  protected void testCreateAndDeleteShard() throws IOException, SolrServerException {
+
+  @Test
+  public void testCreateAndDeleteShard() throws IOException, SolrServerException {
+
     // Create an implicit collection
     String collectionName = "solrj_implicit";
-    CollectionAdminResponse response = new CollectionAdminRequest.Create()
-            .setCollectionName(collectionName)
-            .setShards("shardA,shardB")
-            .setConfigName("conf1")
-            .setRouterName("implicit").process(cloudClient);
+    CollectionAdminResponse response
+        = CollectionAdminRequest.createCollectionWithImplicitRouter(collectionName, "conf", "shardA,shardB", 1)
+        .process(cluster.getSolrClient());
 
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
     Map<String, NamedList<Integer>> coresStatus = response.getCollectionCoresStatus();
     assertEquals(2, coresStatus.size());
 
-    cloudClient.setDefaultCollection(collectionName);
     // Add a shard to the implicit collection
-    response = new CollectionAdminRequest
-        .CreateShard()
-            .setCollectionName(collectionName)
-            .setShardName("shardC").process(cloudClient);
+    response = CollectionAdminRequest.createShard(collectionName, "shardC").process(cluster.getSolrClient());
 
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
@@ -149,57 +113,38 @@ public class CollectionsAPISolrJTest extends AbstractFullDistribZkTestBase {
     assertEquals(1, coresStatus.size());
     assertEquals(0, (int) coresStatus.get(collectionName + "_shardC_replica1").get("status"));
 
-    CollectionAdminRequest.DeleteShard deleteShardRequest = new CollectionAdminRequest
-        .DeleteShard()
-            .setCollectionName(collectionName)
-            .setShardName("shardC");
-    response = deleteShardRequest.process(cloudClient);
+    response = CollectionAdminRequest.deleteShard(collectionName, "shardC").process(cluster.getSolrClient());
 
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
     Map<String, NamedList<Integer>> nodesStatus = response.getCollectionNodesStatus();
     assertEquals(1, nodesStatus.size());
   }
-  
-  protected void testReloadCollection() throws IOException, SolrServerException {
-    cloudClient.setDefaultCollection(DEFAULT_COLLECTION);
-    CollectionAdminRequest.Reload reloadCollectionRequest = new CollectionAdminRequest.Reload()
-            .setCollectionName("collection1");
-    CollectionAdminResponse response = reloadCollectionRequest.process(cloudClient);
 
-    assertEquals(0, response.getStatus());
-  }
-  
-  protected void testCreateAndDeleteAlias() throws IOException, SolrServerException {
-    CollectionAdminRequest.CreateAlias createAliasRequest = new CollectionAdminRequest
-        .CreateAlias()
-            .setAliasName("solrj_alias")
-            .setAliasedCollections(DEFAULT_COLLECTION);
-    CollectionAdminResponse response = createAliasRequest.process(cloudClient);
+  @Test
+  public void testCreateAndDeleteAlias() throws IOException, SolrServerException {
+
+    final String collection = "aliasedCollection";
+    CollectionAdminRequest.createCollection(collection, "conf", 1, 1).process(cluster.getSolrClient());
 
+    CollectionAdminResponse response
+        = CollectionAdminRequest.createAlias("solrj_alias", collection).process(cluster.getSolrClient());
     assertEquals(0, response.getStatus());
 
-    CollectionAdminRequest.DeleteAlias deleteAliasRequest = new CollectionAdminRequest.DeleteAlias()
-            .setAliasName("solrj_alias");
-    deleteAliasRequest.process(cloudClient);
-    
+    response = CollectionAdminRequest.deleteAlias("solrj_alias").process(cluster.getSolrClient());
     assertEquals(0, response.getStatus());
   }
-  
-  protected void testSplitShard() throws Exception {
-    String collectionName = "solrj_test_splitshard";
-    cloudClient.setDefaultCollection(collectionName);
-    
-    CollectionAdminRequest.Create createCollectionRequest = new CollectionAdminRequest.Create()
-            .setConfigName("conf1")
-            .setNumShards(2)
-            .setCollectionName(collectionName);
-    createCollectionRequest.process(cloudClient);
-    
-    CollectionAdminRequest.SplitShard splitShardRequest = new CollectionAdminRequest.SplitShard()
-            .setCollectionName(collectionName)
-            .setShardName("shard1");
-    CollectionAdminResponse response = splitShardRequest.process(cloudClient);
+
+  @Test
+  public void testSplitShard() throws Exception {
+
+    final String collectionName = "solrj_test_splitshard";
+    CollectionAdminRequest.createCollection(collectionName, "conf", 2, 1)
+        .process(cluster.getSolrClient());
+
+    CollectionAdminResponse response = CollectionAdminRequest.splitShard(collectionName)
+        .setShardName("shard1")
+        .process(cluster.getSolrClient());
 
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
@@ -207,267 +152,204 @@ public class CollectionsAPISolrJTest extends AbstractFullDistribZkTestBase {
     assertEquals(0, (int) coresStatus.get(collectionName + "_shard1_0_replica1").get("status"));
     assertEquals(0, (int) coresStatus.get(collectionName + "_shard1_1_replica1").get("status"));
 
-    waitForRecoveriesToFinish(collectionName, false);
-    waitForThingsToLevelOut(10);
+    waitForState("Expected all shards to be active and parent shard to be removed", collectionName, (n, c) -> {
+      if (c.getSlice("shard1").getState() == Slice.State.ACTIVE)
+        return false;
+      for (Replica r : c.getReplicas()) {
+        if (r.isActive(n) == false)
+          return false;
+      }
+      return true;
+    });
     
     // Test splitting using split.key
-    splitShardRequest = new CollectionAdminRequest.SplitShard()
-            .setCollectionName(collectionName)
-            .setSplitKey("b!");
-    response = splitShardRequest.process(cloudClient);
+    response = CollectionAdminRequest.splitShard(collectionName)
+        .setSplitKey("b!")
+        .process(cluster.getSolrClient());
 
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
 
-    waitForRecoveriesToFinish(collectionName, false);
-    waitForThingsToLevelOut(10);
-    
-    ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
-    Collection<Slice> slices = clusterState.getActiveSlices(collectionName);
-    assertEquals("ClusterState: "+ clusterState.getActiveSlices(collectionName), 5, slices.size());  
+    waitForState("Expected 5 slices to be active", collectionName, (n, c) -> c.getActiveSlices().size() == 5);
     
   }
 
-  private void testCreateCollectionWithPropertyParam() throws Exception {
+  @Test
+  public void testCreateCollectionWithPropertyParam() throws Exception {
+
     String collectionName = "solrj_test_core_props";
     
-    File tmpDir = createTempDir("testPropertyParamsForCreate").toFile();
-    File dataDir = new File(tmpDir, "dataDir-" + TestUtil.randomSimpleString(random(), 1, 5));
-    File ulogDir = new File(tmpDir, "ulogDir-" + TestUtil.randomSimpleString(random(), 1, 5));
+    Path tmpDir = createTempDir("testPropertyParamsForCreate");
+    Path dataDir = tmpDir.resolve("dataDir-" + TestUtil.randomSimpleString(random(), 1, 5));
+    Path ulogDir = tmpDir.resolve("ulogDir-" + TestUtil.randomSimpleString(random(), 1, 5));
 
-    Properties properties = new Properties();
-    properties.put(CoreAdminParams.DATA_DIR, dataDir.getAbsolutePath());
-    properties.put(CoreAdminParams.ULOG_DIR, ulogDir.getAbsolutePath());
+    CollectionAdminResponse response = CollectionAdminRequest.createCollection(collectionName, "conf", 1, 1)
+        .withProperty(CoreAdminParams.DATA_DIR, dataDir.toString())
+        .withProperty(CoreAdminParams.ULOG_DIR, ulogDir.toString())
+        .process(cluster.getSolrClient());
 
-    CollectionAdminRequest.Create createReq = new CollectionAdminRequest.Create()
-            .setCollectionName(collectionName)
-            .setNumShards(1)
-            .setConfigName("conf1")
-            .setProperties(properties);
-
-    CollectionAdminResponse response = createReq.process(cloudClient);
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
     Map<String, NamedList<Integer>> coresStatus = response.getCollectionCoresStatus();
     assertEquals(1, coresStatus.size());
 
-    DocCollection testCollection = cloudClient.getZkStateReader()
-        .getClusterState().getCollection(collectionName);
+    DocCollection testCollection = getCollectionState(collectionName);
 
     Replica replica1 = testCollection.getReplica("core_node1");
+    CoreStatus coreStatus = getCoreStatus(replica1);
 
-    try (HttpSolrClient client = getHttpSolrClient(replica1.getStr("base_url"))) {
-      CoreAdminResponse status = CoreAdminRequest.getStatus(replica1.getStr("core"), client);
-      NamedList<Object> coreStatus = status.getCoreStatus(replica1.getStr("core"));
-      String dataDirStr = (String) coreStatus.get("dataDir");
-      assertEquals("Data dir does not match param given in property.dataDir syntax",
-          new File(dataDirStr).getAbsolutePath(), dataDir.getAbsolutePath());
-    }
+    assertEquals(Paths.get(coreStatus.getDataDirectory()).toString(), dataDir.toString());
 
-    CollectionAdminRequest.Delete deleteCollectionRequest = new CollectionAdminRequest.Delete();
-    deleteCollectionRequest.setCollectionName(collectionName);
-    deleteCollectionRequest.process(cloudClient);
   }
 
-  private void testAddAndDeleteReplica() throws Exception {
-    String collectionName = "solrj_replicatests";
-    createCollection(collectionName, cloudClient, 1, 2);
+  @Test
+  public void testAddAndDeleteReplica() throws Exception {
 
-    cloudClient.setDefaultCollection(collectionName);
+    final String collectionName = "solrj_replicatests";
+    CollectionAdminRequest.createCollection(collectionName, "conf", 1, 2)
+        .process(cluster.getSolrClient());
 
-    String newReplicaName = Assign.assignNode(cloudClient.getZkStateReader().getClusterState().getCollection(collectionName));
-    ArrayList<String> nodeList = new ArrayList<>(cloudClient.getZkStateReader().getClusterState().getLiveNodes());
+    String newReplicaName = Assign.assignNode(getCollectionState(collectionName));
+    ArrayList<String> nodeList
+        = new ArrayList<>(cluster.getSolrClient().getZkStateReader().getClusterState().getLiveNodes());
     Collections.shuffle(nodeList, random());
-    CollectionAdminRequest.AddReplica addReplica = new CollectionAdminRequest.AddReplica()
-            .setCollectionName(collectionName)
-            .setShardName("shard1")
-            .setNode(nodeList.get(0));
-    CollectionAdminResponse response = addReplica.process(cloudClient);
+    final String node = nodeList.get(0);
+
+    CollectionAdminResponse response = CollectionAdminRequest.addReplicaToShard(collectionName, "shard1")
+        .setNode(node)
+        .process(cluster.getSolrClient());
 
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
 
-    TimeOut timeout = new TimeOut(3, TimeUnit.SECONDS);
-    Replica newReplica = null;
-
-    while (! timeout.hasTimedOut() && newReplica == null) {
-      Slice slice = cloudClient.getZkStateReader().getClusterState().getSlice(collectionName, "shard1");
-      newReplica = slice.getReplica(newReplicaName);
-    }
-
-    assertNotNull(newReplica);
-
-    assertEquals("Replica should be created on the right node",
-        cloudClient.getZkStateReader().getBaseUrlForNodeName(nodeList.get(0)),
-        newReplica.getStr(ZkStateReader.BASE_URL_PROP)
-    );
+    waitForState("Expected to see replica " + newReplicaName + " on node " + node, collectionName, (n, c) -> {
+      Replica r = c.getSlice("shard1").getReplica(newReplicaName);
+      return r != null && r.getNodeName().equals(node);
+    });
     
     // Test DELETEREPLICA
-    CollectionAdminRequest.DeleteReplica deleteReplicaRequest = new CollectionAdminRequest.DeleteReplica()
-            .setCollectionName(collectionName)
-            .setShardName("shard1")
-            .setReplica(newReplicaName);
-    response = deleteReplicaRequest.process(cloudClient);
-
+    response = CollectionAdminRequest.deleteReplica(collectionName, "shard1", newReplicaName)
+        .process(cluster.getSolrClient());
     assertEquals(0, response.getStatus());
 
-    timeout = new TimeOut(3, TimeUnit.SECONDS);
-
-    while (! timeout.hasTimedOut() && newReplica != null) {
-      Slice slice = cloudClient.getZkStateReader().getClusterState().getSlice(collectionName, "shard1");
-      newReplica = slice.getReplica(newReplicaName);
-    }
+    waitForState("Expected replica " + newReplicaName + " to vanish from cluster state", collectionName,
+        (n, c) -> c.getSlice("shard1").getReplica(newReplicaName) == null);
 
-    assertNull(newReplica);
   }
 
-  private void testClusterProp() throws InterruptedException, IOException, SolrServerException {
-    CollectionAdminRequest.ClusterProp clusterPropRequest = new CollectionAdminRequest.ClusterProp()
-            .setPropertyName(ZkStateReader.LEGACY_CLOUD)
-            .setPropertyValue("false");
-    CollectionAdminResponse response = clusterPropRequest.process(cloudClient);
+  @Test
+  public void testClusterProp() throws InterruptedException, IOException, SolrServerException {
+
+    CollectionAdminResponse response = CollectionAdminRequest.setClusterProperty(ZkStateReader.LEGACY_CLOUD, "false")
+        .process(cluster.getSolrClient());
 
     assertEquals(0, response.getStatus());
 
-    TimeOut timeout = new TimeOut(3, TimeUnit.SECONDS);
-    boolean changed = false;
-    
-    while(! timeout.hasTimedOut()){
-      Thread.sleep(10);
-      changed = Objects.equals("false",
-          cloudClient.getZkStateReader().getClusterProperty(ZkStateReader.LEGACY_CLOUD, "none"));
-      if(changed) break;
-    }
-    assertTrue("The Cluster property wasn't set", changed);
+    ClusterProperties props = new ClusterProperties(zkClient());
+    assertEquals("Cluster property was not set", props.getClusterProperty(ZkStateReader.LEGACY_CLOUD, "true"), "false");
     
     // Unset ClusterProp that we set.
-    clusterPropRequest = new CollectionAdminRequest.ClusterProp()
-            .setPropertyName(ZkStateReader.LEGACY_CLOUD)
-            .setPropertyValue(null);
-    clusterPropRequest.process(cloudClient);
-
-    timeout = new TimeOut(3, TimeUnit.SECONDS);
-    changed = false;
-    while(! timeout.hasTimedOut()) {
-      Thread.sleep(10);
-      changed = (cloudClient.getZkStateReader().getClusterProperty(ZkStateReader.LEGACY_CLOUD, (String) null) == null);
-      if(changed)  
-        break;
-    }
-    assertTrue("The Cluster property wasn't unset", changed);
+    CollectionAdminRequest.setClusterProperty(ZkStateReader.LEGACY_CLOUD, null).process(cluster.getSolrClient());
+    assertEquals("Cluster property was not unset", props.getClusterProperty(ZkStateReader.LEGACY_CLOUD, "true"), "true");
+
   }
 
-  private void testAddAndRemoveRole() throws InterruptedException, IOException, SolrServerException {
-    cloudClient.setDefaultCollection(DEFAULT_COLLECTION);
-    Replica replica = cloudClient.getZkStateReader().getLeaderRetry(DEFAULT_COLLECTION, SHARD1);
-    CollectionAdminRequest.AddRole addRoleRequest = new CollectionAdminRequest.AddRole()
-            .setNode(replica.getNodeName())
-            .setRole("overseer");
-    addRoleRequest.process(cloudClient);
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testAddAndRemoveRole() throws InterruptedException, IOException, SolrServerException {
+
+    String node = cluster.getRandomJetty(random()).getNodeName();
+
+    CollectionAdminRequest.addRole(node, "overseer").process(cluster.getSolrClient());
 
-    CollectionAdminRequest.ClusterStatus clusterStatusRequest = new CollectionAdminRequest.ClusterStatus()
-            .setCollectionName(DEFAULT_COLLECTION);
-    CollectionAdminResponse response = clusterStatusRequest.process(cloudClient);
+    CollectionAdminResponse response = CollectionAdminRequest.getClusterStatus().process(cluster.getSolrClient());
 
     NamedList<Object> rsp = response.getResponse();
-    NamedList<Object> cluster = (NamedList<Object>) rsp.get("cluster");
-    assertNotNull("Cluster state should not be null", cluster);
-    Map<String, Object> roles = (Map<String, Object>) cluster.get("roles");
+    NamedList<Object> cs = (NamedList<Object>) rsp.get("cluster");
+    assertNotNull("Cluster state should not be null", cs);
+    Map<String, Object> roles = (Map<String, Object>) cs.get("roles");
     assertNotNull("Role information should not be null", roles);
     List<String> overseer = (List<String>) roles.get("overseer");
     assertNotNull(overseer);
     assertEquals(1, overseer.size());
-    assertTrue(overseer.contains(replica.getNodeName()));
+    assertTrue(overseer.contains(node));
     
     // Remove role
-    new CollectionAdminRequest.RemoveRole()
-            .setNode(replica.getNodeName())
-            .setRole("overseer")
-            .process(cloudClient);
-
-    clusterStatusRequest = new CollectionAdminRequest.ClusterStatus();
-    clusterStatusRequest.setCollectionName(DEFAULT_COLLECTION);
-    response = clusterStatusRequest.process(cloudClient);
+    CollectionAdminRequest.removeRole(node, "overseer").process(cluster.getSolrClient());
 
+    response = CollectionAdminRequest.getClusterStatus().process(cluster.getSolrClient());
     rsp = response.getResponse();
-    cluster = (NamedList<Object>) rsp.get("cluster");
-    assertNotNull("Cluster state should not be null", cluster);
-    roles = (Map<String, Object>) cluster.get("roles");
+    cs = (NamedList<Object>) rsp.get("cluster");
+    assertNotNull("Cluster state should not be null", cs);
+    roles = (Map<String, Object>) cs.get("roles");
     assertNotNull("Role information should not be null", roles);
     overseer = (List<String>) roles.get("overseer");
-    assertFalse(overseer.contains(replica.getNodeName()));
+    assertFalse(overseer.contains(node));
   }
-  
-  private void testOverseerStatus() throws IOException, SolrServerException {
-    CollectionAdminResponse response = new CollectionAdminRequest.OverseerStatus().process(cloudClient);
+
+  @Test
+  public void testOverseerStatus() throws IOException, SolrServerException {
+    CollectionAdminResponse response = new CollectionAdminRequest.OverseerStatus().process(cluster.getSolrClient());
     assertEquals(0, response.getStatus());
     assertNotNull("overseer_operations shouldn't be null", response.getResponse().get("overseer_operations"));
   }
-  
-  private void testList() throws IOException, SolrServerException {
-    CollectionAdminResponse response = new CollectionAdminRequest.List().process(cloudClient);
+
+  @Test
+  public void testList() throws IOException, SolrServerException {
+    CollectionAdminResponse response = new CollectionAdminRequest.List().process(cluster.getSolrClient());
     assertEquals(0, response.getStatus());
     assertNotNull("collection list should not be null", response.getResponse().get("collections"));
   }
-  
-  private void testAddAndDeleteReplicaProp() throws InterruptedException, IOException, SolrServerException {
-    Replica replica = cloudClient.getZkStateReader().getLeaderRetry(DEFAULT_COLLECTION, SHARD1);
-    CollectionAdminResponse response = new CollectionAdminRequest.AddReplicaProp()
-            .setCollectionName(DEFAULT_COLLECTION)
-            .setShardName(SHARD1)
-            .setReplica(replica.getName())
-            .setPropertyName("preferredleader")
-            .setPropertyValue("true").process(cloudClient);
-    assertEquals(0, response.getStatus());
 
-    TimeOut timeout = new TimeOut(20, TimeUnit.SECONDS);
-    String propertyValue = null;
-    
-    String replicaName = replica.getName();
-    while (! timeout.hasTimedOut()) {
-      ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
-      replica = clusterState.getReplica(DEFAULT_COLLECTION, replicaName);
-      propertyValue = replica.getStr("property.preferredleader"); 
-      if(StringUtils.equals("true", propertyValue))
-        break;
-      Thread.sleep(50);
-    }
-    
-    assertEquals("Replica property was not updated, Latest value: " +
-        cloudClient.getZkStateReader().getClusterState().getReplica(DEFAULT_COLLECTION, replicaName),
-        "true",
-        propertyValue);
-
-    response = new CollectionAdminRequest.DeleteReplicaProp()
-            .setCollectionName(DEFAULT_COLLECTION)
-            .setShardName(SHARD1)
-            .setReplica(replicaName)
-            .setPropertyName("property.preferredleader").process(cloudClient);
+  @Test
+  public void testAddAndDeleteReplicaProp() throws InterruptedException, IOException, SolrServerException {
+
+    final String collection = "replicaProperties";
+    CollectionAdminRequest.createCollection(collection, "conf", 2, 2)
+        .process(cluster.getSolrClient());
+
+    final Replica replica = getCollectionState(collection).getLeader("shard1");
+    CollectionAdminResponse response
+        = CollectionAdminRequest.addReplicaProperty(collection, "shard1", replica.getName(), "preferredleader", "true")
+        .process(cluster.getSolrClient());
     assertEquals(0, response.getStatus());
 
-    timeout = new TimeOut(20, TimeUnit.SECONDS);
-    boolean updated = false;
+    waitForState("Expecting property 'preferredleader' to appear on replica " + replica.getName(), collection,
+        (n, c) -> "true".equals(c.getReplica(replica.getName()).getStr("property.preferredleader")));
 
-    while (! timeout.hasTimedOut()) {
-      ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
-      replica = clusterState.getReplica(DEFAULT_COLLECTION, replicaName);
-      updated = replica.getStr("property.preferredleader") == null;
-      if(updated)
-        break;
-      Thread.sleep(50);
-    }
+    response = CollectionAdminRequest.deleteReplicaProperty(collection, "shard1", replica.getName(), "property.preferredleader")
+        .process(cluster.getSolrClient());
+    assertEquals(0, response.getStatus());
 
-    assertTrue("Replica property was not removed", updated);
+    waitForState("Expecting property 'preferredleader' to be removed from replica " + replica.getName(), collection,
+        (n, c) -> c.getReplica(replica.getName()).getStr("property.preferredleader") == null);
     
   }
-  
-  private void testBalanceShardUnique() throws IOException,
+
+  @Test
+  public void testBalanceShardUnique() throws IOException,
       SolrServerException, KeeperException, InterruptedException {
-    CollectionAdminResponse response = new CollectionAdminRequest.BalanceShardUnique()
-            .setCollection(DEFAULT_COLLECTION)
-            .setPropertyName("preferredLeader").process(cloudClient);
+
+    final String collection = "balancedProperties";
+    CollectionAdminRequest.createCollection(collection, "conf", 2, 2)
+        .process(cluster.getSolrClient());
+
+    CollectionAdminResponse response = CollectionAdminRequest.balanceReplicaProperty(collection, "preferredLeader")
+        .process(cluster.getSolrClient());
     assertEquals(0, response.getStatus());
 
-    verifyUniqueAcrossCollection(cloudClient, DEFAULT_COLLECTION, "property.preferredleader");    
+    waitForState("Expecting 'preferredleader' property to be balanced across all shards", collection, (n, c) -> {
+      for (Slice slice : c) {
+        int count = 0;
+        for (Replica replica : slice) {
+          if ("true".equals(replica.getStr("property.preferredleader")))
+            count += 1;
+        }
+        if (count != 1)
+          return false;
+      }
+      return true;
+    });
+
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12aff1cf/solr/core/src/test/org/apache/solr/cloud/DeleteLastCustomShardedReplicaTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/DeleteLastCustomShardedReplicaTest.java b/solr/core/src/test/org/apache/solr/cloud/DeleteLastCustomShardedReplicaTest.java
index dcc99a4..c46362e 100644
--- a/solr/core/src/test/org/apache/solr/cloud/DeleteLastCustomShardedReplicaTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/DeleteLastCustomShardedReplicaTest.java
@@ -16,106 +16,40 @@
  */
 package org.apache.solr.cloud;
 
-import org.apache.solr.client.solrj.SolrRequest;
-import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.client.solrj.impl.CloudSolrClient;
-import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.common.cloud.DocCollection;
-import org.apache.solr.common.cloud.ImplicitDocRouter;
 import org.apache.solr.common.cloud.Replica;
-import org.apache.solr.common.cloud.Slice;
-import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.params.MapSolrParams;
-import org.apache.solr.common.params.SolrParams;
-import org.apache.solr.common.util.Utils;
-import org.apache.solr.util.TimeOut;
+import org.junit.BeforeClass;
 import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
+public class DeleteLastCustomShardedReplicaTest extends SolrCloudTestCase {
 
-import static org.apache.solr.cloud.OverseerCollectionMessageHandler.NUM_SLICES;
-import static org.apache.solr.cloud.OverseerCollectionMessageHandler.SHARDS_PROP;
-import static org.apache.solr.common.util.Utils.makeMap;
-import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETEREPLICA;
-
-public class DeleteLastCustomShardedReplicaTest extends AbstractFullDistribZkTestBase {
-
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-  protected String getSolrXml() {
-    return "solr.xml";
-  }
-
-  public DeleteLastCustomShardedReplicaTest() {
-    sliceCount = 2;
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(2)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
   }
 
   @Test
-  @ShardsFixed(num = 2)
   public void test() throws Exception {
-    try (CloudSolrClient client = createCloudClient(null))  {
-      int replicationFactor = 1;
-      int maxShardsPerNode = 5;
-
-      Map<String, Object> props = Utils.makeMap(
-          "router.name", ImplicitDocRouter.NAME,
-          ZkStateReader.REPLICATION_FACTOR, replicationFactor,
-          ZkStateReader.MAX_SHARDS_PER_NODE, maxShardsPerNode,
-          NUM_SLICES, 1,
-          SHARDS_PROP, "a,b");
 
-      Map<String,List<Integer>> collectionInfos = new HashMap<>();
+    final String collectionName = "customcollreplicadeletion";
 
-      String collectionName = "customcollreplicadeletion";
+    CollectionAdminRequest.createCollectionWithImplicitRouter(collectionName, "conf", "a,b", 1)
+        .setMaxShardsPerNode(5)
+        .process(cluster.getSolrClient());
 
-      createCollection(collectionInfos, collectionName, props, client);
+    DocCollection collectionState = getCollectionState(collectionName);
+    Replica replica = getRandomReplica(collectionState.getSlice("a"));
 
-      waitForRecoveriesToFinish(collectionName, false);
+    CollectionAdminRequest.deleteReplica(collectionName, "a", replica.getName())
+        .process(cluster.getSolrClient());
 
-      DocCollection testcoll = getCommonCloudSolrClient().getZkStateReader()
-              .getClusterState().getCollection(collectionName);
-      Replica replica = testcoll.getSlice("a").getReplicas().iterator().next();
-
-      removeAndWaitForReplicaGone(client, collectionName, replica, "a", replicationFactor-1);
-    }
-  }
+    waitForState("Expected shard 'a' to have no replicas", collectionName, (n, c) -> {
+      return c.getSlice("a") == null || c.getSlice("a").getReplicas().size() == 0;
+    });
 
-  protected void removeAndWaitForReplicaGone(CloudSolrClient client, String COLL_NAME, Replica replica, String shard,
-      final int expectedNumReplicasRemaining)
-      throws SolrServerException, IOException, InterruptedException {
-    Map m = makeMap("collection", COLL_NAME, "action", DELETEREPLICA.toLower(), "shard",
-        shard, "replica", replica.getName());
-    SolrParams params = new MapSolrParams(m);
-    SolrRequest request = new QueryRequest(params);
-    request.setPath("/admin/collections");
-    client.request(request);
-    TimeOut timeout = new TimeOut(3, TimeUnit.SECONDS);
-    boolean success = false;
-    DocCollection testcoll = null;
-    while (! timeout.hasTimedOut()) {
-      testcoll = getCommonCloudSolrClient().getZkStateReader()
-          .getClusterState().getCollection(COLL_NAME);
-      // As of SOLR-5209 the last replica deletion no longer leads to
-      // the deletion of the slice.
-      final Slice slice = testcoll.getSlice(shard);
-      final int actualNumReplicasRemaining = (slice == null ? 0 : slice.getReplicas().size());
-      success = (actualNumReplicasRemaining == expectedNumReplicasRemaining);
-      if (success) {
-        log.info("replica cleaned up {}/{} core {}",
-            shard + "/" + replica.getName(), replica.getStr("core"));
-        log.info("current state {}", testcoll);
-        break;
-      }
-      Thread.sleep(100);
-    }
-    assertTrue("Replica not cleaned up", success);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12aff1cf/solr/core/src/test/org/apache/solr/cloud/DeleteShardTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/DeleteShardTest.java b/solr/core/src/test/org/apache/solr/cloud/DeleteShardTest.java
index 9097363..ed3d03b 100644
--- a/solr/core/src/test/org/apache/solr/cloud/DeleteShardTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/DeleteShardTest.java
@@ -20,202 +20,123 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.client.solrj.request.CoreAdminRequest;
-import org.apache.solr.client.solrj.request.QueryRequest;
-import org.apache.solr.client.solrj.response.CollectionAdminResponse;
-import org.apache.solr.client.solrj.response.CoreAdminResponse;
+import org.apache.solr.client.solrj.request.CoreStatus;
 import org.apache.solr.cloud.overseer.OverseerAction;
-import org.apache.solr.common.SolrException;
-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.Slice.State;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.params.CollectionParams;
-import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.FileUtils;
 import org.apache.zookeeper.KeeperException;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
-public class DeleteShardTest extends AbstractFullDistribZkTestBase {
-
-  public DeleteShardTest() {
-    super();
-    sliceCount = 2;
-  }
+public class DeleteShardTest extends SolrCloudTestCase {
 
   // TODO: Custom hash slice deletion test
 
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(2)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
+  }
+
   @Test
-  @ShardsFixed(num = 2)
   public void test() throws Exception {
-    ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
 
-    Slice slice1 = clusterState.getSlice(AbstractDistribZkTestBase.DEFAULT_COLLECTION, SHARD1);
-    Slice slice2 = clusterState.getSlice(AbstractDistribZkTestBase.DEFAULT_COLLECTION, SHARD2);
+    final String collection = "deleteShard";
 
-    assertNotNull("Shard1 not found", slice1);
-    assertNotNull("Shard2 not found", slice2);
-    assertSame("Shard1 is not active", Slice.State.ACTIVE, slice1.getState());
-    assertSame("Shard2 is not active", Slice.State.ACTIVE, slice2.getState());
+    CollectionAdminRequest.createCollection(collection, "conf", 2, 1)
+        .process(cluster.getSolrClient());
 
-    try {
-      deleteShard(SHARD1);
-      fail("Deleting an active shard should not have succeeded");
-    } catch (HttpSolrClient.RemoteSolrException e) {
-      // expected
-    }
+    DocCollection state = getCollectionState(collection);
+    assertEquals(State.ACTIVE, state.getSlice("shard1").getState());
+    assertEquals(State.ACTIVE, state.getSlice("shard2").getState());
 
-    setSliceState(SHARD1, Slice.State.INACTIVE);
+    // Can't delete an ACTIVE shard
+    expectThrows(Exception.class, () -> {
+      CollectionAdminRequest.deleteShard(collection, "shard1").process(cluster.getSolrClient());
+    });
 
-    clusterState = cloudClient.getZkStateReader().getClusterState();
+    setSliceState(collection, "shard1", Slice.State.INACTIVE);
 
-    slice1 = clusterState.getSlice(AbstractDistribZkTestBase.DEFAULT_COLLECTION, SHARD1);
+    // Can delete an INATIVE shard
+    CollectionAdminRequest.deleteShard(collection, "shard1").process(cluster.getSolrClient());
+    waitForState("Expected 'shard1' to be removed", collection, (n, c) -> {
+      return c.getSlice("shard1") == null;
+    });
 
-    assertSame("Shard1 is not inactive yet.", Slice.State.INACTIVE, slice1.getState());
-
-    deleteShard(SHARD1);
-
-    confirmShardDeletion(SHARD1);
-
-    setSliceState(SHARD2, Slice.State.CONSTRUCTION);
-    deleteShard(SHARD2);
-    confirmShardDeletion(SHARD2);
-  }
+    // Can delete a shard under construction
+    setSliceState(collection, "shard2", Slice.State.CONSTRUCTION);
+    CollectionAdminRequest.deleteShard(collection, "shard2").process(cluster.getSolrClient());
+    waitForState("Expected 'shard2' to be removed", collection, (n, c) -> {
+      return c.getSlice("shard2") == null;
+    });
 
-  protected void confirmShardDeletion(String shard) throws SolrServerException, KeeperException,
-      InterruptedException {
-    ZkStateReader zkStateReader = cloudClient.getZkStateReader();
-    ClusterState clusterState = zkStateReader.getClusterState();
-    int counter = 10;
-    while (counter-- > 0) {
-      clusterState = zkStateReader.getClusterState();
-      if (clusterState.getSlice("collection1", shard) == null) {
-        break;
-      }
-      Thread.sleep(1000);
-    }
-
-    assertNull("Cluster still contains shard1 even after waiting for it to be deleted.",
-        clusterState.getSlice(AbstractDistribZkTestBase.DEFAULT_COLLECTION, SHARD1));
   }
 
-  protected void deleteShard(String shard) throws SolrServerException, IOException,
+  protected void setSliceState(String collection, String slice, State state) throws SolrServerException, IOException,
       KeeperException, InterruptedException {
 
-    ModifiableSolrParams params = new ModifiableSolrParams();
-    params.set("action", CollectionParams.CollectionAction.DELETESHARD.toString());
-    params.set("collection", AbstractFullDistribZkTestBase.DEFAULT_COLLECTION);
-    params.set("shard", shard);
-    SolrRequest request = new QueryRequest(params);
-    request.setPath("/admin/collections");
-
-    String baseUrl = ((HttpSolrClient) shardToJetty.get(SHARD1).get(0).client.solrClient)
-        .getBaseURL();
-    baseUrl = baseUrl.substring(0, baseUrl.length() - "collection1".length());
-
-    try (HttpSolrClient baseServer = getHttpSolrClient(baseUrl)) {
-      baseServer.setConnectionTimeout(15000);
-      baseServer.setSoTimeout(60000);
-      baseServer.request(request);
-    }
-  }
+    CloudSolrClient client = cluster.getSolrClient();
 
-  protected void setSliceState(String slice, State state) throws SolrServerException, IOException,
-      KeeperException, InterruptedException {
-    DistributedQueue inQueue = Overseer.getStateUpdateQueue(cloudClient.getZkStateReader().getZkClient());
+    // TODO can this be encapsulated better somewhere?
+    DistributedQueue inQueue = Overseer.getStateUpdateQueue(client.getZkStateReader().getZkClient());
     Map<String, Object> propMap = new HashMap<>();
     propMap.put(Overseer.QUEUE_OPERATION, OverseerAction.UPDATESHARDSTATE.toLower());
     propMap.put(slice, state.toString());
-    propMap.put(ZkStateReader.COLLECTION_PROP, "collection1");
+    propMap.put(ZkStateReader.COLLECTION_PROP, collection);
     ZkNodeProps m = new ZkNodeProps(propMap);
-    ZkStateReader zkStateReader = cloudClient.getZkStateReader();
     inQueue.offer(Utils.toJSON(m));
-    boolean transition = false;
-
-    for (int counter = 10; counter > 0; counter--) {
-      ClusterState clusterState = zkStateReader.getClusterState();
-      State sliceState = clusterState.getSlice("collection1", slice).getState();
-      if (sliceState == state) {
-        transition = true;
-        break;
-      }
-      Thread.sleep(1000);
-    }
-
-    if (!transition) {
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Could not set shard [" + slice + "] as " + state);
-    }
+
+    waitForState("Expected shard " + slice + " to be in state " + state.toString(), collection, (n, c) -> {
+      return c.getSlice(slice).getState() == state;
+    });
+
   }
 
   @Test
   public void testDirectoryCleanupAfterDeleteShard() throws InterruptedException, IOException, SolrServerException {
-    CollectionAdminResponse rsp = new CollectionAdminRequest.Create()
-        .setCollectionName("deleteshard_test")
-        .setRouterName("implicit")
-        .setShards("a,b,c")
-        .setReplicationFactor(1)
-        .setConfigName("conf1")
-        .process(cloudClient);
+
+    final String collection = "deleteshard_test";
+    CollectionAdminRequest.createCollectionWithImplicitRouter(collection, "conf", "a,b,c", 1)
+        .setMaxShardsPerNode(2)
+        .process(cluster.getSolrClient());
 
     // Get replica details
-    Replica leader = cloudClient.getZkStateReader().getLeaderRetry("deleteshard_test", "a");
-    String baseUrl = (String) leader.get("base_url");
-    String core = (String) leader.get("core");
+    Replica leader = getCollectionState(collection).getLeader("a");
 
-    String instanceDir;
-    String dataDir;
+    CoreStatus coreStatus = getCoreStatus(leader);
+    assertTrue("Instance directory doesn't exist", FileUtils.fileExists(coreStatus.getInstanceDirectory()));
+    assertTrue("Data directory doesn't exist", FileUtils.fileExists(coreStatus.getDataDirectory()));
 
-    try (HttpSolrClient client = getHttpSolrClient(baseUrl)) {
-      CoreAdminResponse statusResp = CoreAdminRequest.getStatus(core, client);
-      NamedList r = statusResp.getCoreStatus().get(core);
-      instanceDir = (String) r.findRecursive("instanceDir");
-      dataDir = (String) r.get("dataDir");
-    }
+    assertEquals(3, getCollectionState(collection).getActiveSlices().size());
 
-    assertTrue("Instance directory doesn't exist", FileUtils.fileExists(instanceDir));
-    assertTrue("Data directory doesn't exist", FileUtils.fileExists(dataDir));
+    // Delete shard 'a'
+    CollectionAdminRequest.deleteShard(collection, "a").process(cluster.getSolrClient());
 
-    assertEquals(3, cloudClient.getZkStateReader().getClusterState().getActiveSlices("deleteshard_test").size());
+    assertEquals(2, getCollectionState(collection).getActiveSlices().size());
+    assertFalse("Instance directory still exists", FileUtils.fileExists(coreStatus.getInstanceDirectory()));
+    assertFalse("Data directory still exists", FileUtils.fileExists(coreStatus.getDataDirectory()));
 
-    // Delete shard 'a'
-    new CollectionAdminRequest.DeleteShard()
-        .setCollectionName("deleteshard_test")
-        .setShardName("a")
-        .process(cloudClient);
-
-    assertEquals(2, cloudClient.getZkStateReader().getClusterState().getActiveSlices("deleteshard_test").size());
-    assertFalse("Instance directory still exists", FileUtils.fileExists(instanceDir));
-    assertFalse("Data directory still exists", FileUtils.fileExists(dataDir));
-
-    leader = cloudClient.getZkStateReader().getLeaderRetry("deleteshard_test", "b");
-    baseUrl = (String) leader.get("base_url");
-    core = (String) leader.get("core");
-
-    try (HttpSolrClient client = getHttpSolrClient(baseUrl)) {
-      CoreAdminResponse statusResp = CoreAdminRequest.getStatus(core, client);
-      NamedList r = statusResp.getCoreStatus().get(core);
-      instanceDir = (String) r.findRecursive("instanceDir");
-      dataDir = (String) r.get("dataDir");
-    }
+    leader = getCollectionState(collection).getLeader("b");
+    coreStatus = getCoreStatus(leader);
 
     // Delete shard 'b'
-    new CollectionAdminRequest.DeleteShard()
-        .setCollectionName("deleteshard_test")
-        .setShardName("b")
+    CollectionAdminRequest.deleteShard(collection, "b")
         .setDeleteDataDir(false)
         .setDeleteInstanceDir(false)
-        .process(cloudClient);
+        .process(cluster.getSolrClient());
 
-    assertEquals(1, cloudClient.getZkStateReader().getClusterState().getActiveSlices("deleteshard_test").size());
-    assertTrue("Instance directory still exists", FileUtils.fileExists(instanceDir));
-    assertTrue("Data directory still exists", FileUtils.fileExists(dataDir));
+    assertEquals(1, getCollectionState(collection).getActiveSlices().size());
+    assertTrue("Instance directory still exists", FileUtils.fileExists(coreStatus.getInstanceDirectory()));
+    assertTrue("Data directory still exists", FileUtils.fileExists(coreStatus.getDataDirectory()));
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12aff1cf/solr/core/src/test/org/apache/solr/cloud/OverseerModifyCollectionTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/OverseerModifyCollectionTest.java b/solr/core/src/test/org/apache/solr/cloud/OverseerModifyCollectionTest.java
index e902ab4..c9a90a5 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OverseerModifyCollectionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OverseerModifyCollectionTest.java
@@ -17,78 +17,62 @@
 
 package org.apache.solr.cloud;
 
-import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
-import static org.apache.solr.common.params.CommonParams.COLLECTIONS_HANDLER_PATH;
-
-import java.lang.invoke.MethodHandles;
 import java.util.Map;
 
-import org.apache.solr.client.solrj.SolrClient;
-import org.apache.solr.client.solrj.impl.HttpSolrClient;
-import org.apache.solr.client.solrj.impl.HttpSolrClient.RemoteSolrException;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.client.solrj.request.ConfigSetAdminRequest;
 import org.apache.solr.client.solrj.request.GenericSolrRequest;
-import org.apache.solr.client.solrj.response.CollectionAdminResponse;
-import org.apache.solr.client.solrj.response.ConfigSetAdminResponse;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.Utils;
 import org.apache.zookeeper.KeeperException;
+import org.junit.BeforeClass;
 import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-public class OverseerModifyCollectionTest extends AbstractFullDistribZkTestBase {
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  
+import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
+import static org.apache.solr.common.params.CommonParams.COLLECTIONS_HANDLER_PATH;
+
+public class OverseerModifyCollectionTest extends SolrCloudTestCase {
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(2)
+        .addConfig("conf1", configset("cloud-minimal"))
+        .addConfig("conf2", configset("cloud-minimal"))
+        .configure();
+  }
+
   @Test
   public void testModifyColl() throws Exception {
-    String collName = "modifyColl";
-    String newConfName = "conf" + random().nextInt();
-    String oldConfName = "conf1";
-    try (SolrClient client = createNewSolrClient("", getBaseUrl((HttpSolrClient) clients.get(0)))) {
-      CollectionAdminResponse rsp;
-      CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collName, oldConfName, 1, 2);
-      rsp = create.process(client);
-      assertEquals(0, rsp.getStatus());
-      assertTrue(rsp.isSuccess());
-      
-      ConfigSetAdminRequest.Create createConfig = new ConfigSetAdminRequest.Create()
-        .setBaseConfigSetName(oldConfName)
-        .setConfigSetName(newConfName);
-      
-      ConfigSetAdminResponse configRsp = createConfig.process(client);
-      
-      assertEquals(0, configRsp.getStatus());
-      
-      ModifiableSolrParams p = new ModifiableSolrParams();
-      p.add("collection", collName);
-      p.add("action", "MODIFYCOLLECTION");
-      p.add("collection.configName", newConfName);
-      client.request(new GenericSolrRequest(POST, COLLECTIONS_HANDLER_PATH, p));
-    }
-    
-    assertEquals(newConfName, getConfigNameFromZk(collName));    
+
+    final String collName = "modifyColl";
+
+    CollectionAdminRequest.createCollection(collName, "conf1", 1, 2)
+        .process(cluster.getSolrClient());
+
+    // TODO create a modifyCollection() method on CollectionAdminRequest
+    ModifiableSolrParams p1 = new ModifiableSolrParams();
+    p1.add("collection", collName);
+    p1.add("action", "MODIFYCOLLECTION");
+    p1.add("collection.configName", "conf2");
+    cluster.getSolrClient().request(new GenericSolrRequest(POST, COLLECTIONS_HANDLER_PATH, p1));
+
+    assertEquals("conf2", getConfigNameFromZk(collName));
     
     //Try an invalid config name
-    try (SolrClient client = createNewSolrClient("", getBaseUrl((HttpSolrClient) clients.get(0)))) {
-      ModifiableSolrParams p = new ModifiableSolrParams();
-      p.add("collection", collName);
-      p.add("action", "MODIFYCOLLECTION");
-      p.add("collection.configName", "notARealConfigName");
-      try{
-        client.request(new GenericSolrRequest(POST, COLLECTIONS_HANDLER_PATH, p));
-        fail("Exception should be thrown");
-      } catch(RemoteSolrException e) {
-        assertTrue(e.getMessage(), e.getMessage().contains("Can not find the specified config set"));
-      }
-    }
+    ModifiableSolrParams p2 = new ModifiableSolrParams();
+    p2.add("collection", collName);
+    p2.add("action", "MODIFYCOLLECTION");
+    p2.add("collection.configName", "notARealConfigName");
+    Exception e = expectThrows(Exception.class, () -> {
+      cluster.getSolrClient().request(new GenericSolrRequest(POST, COLLECTIONS_HANDLER_PATH, p2));
+    });
+
+    assertTrue(e.getMessage(), e.getMessage().contains("Can not find the specified config set"));
 
   }
   
   private String getConfigNameFromZk(String collName) throws KeeperException, InterruptedException {
-    byte[] b = cloudClient.getZkStateReader().getZkClient().getData(ZkStateReader.getCollectionPathRoot(collName), null, null, false);
+    byte[] b = zkClient().getData(ZkStateReader.getCollectionPathRoot(collName), null, null, false);
     Map confData = (Map) Utils.fromJSON(b);
     return (String) confData.get(ZkController.CONFIGNAME_PROP); 
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12aff1cf/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java b/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java
index dec54d9..762bbeb 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java
@@ -16,118 +16,95 @@
  */
 package org.apache.solr.cloud;
 
-import org.apache.lucene.util.LuceneTestCase;
-import org.apache.solr.SolrTestCaseJ4.SuppressSSL;
-import org.apache.solr.client.solrj.SolrRequest;
+import java.lang.invoke.MethodHandles;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
-import org.apache.solr.client.solrj.impl.CloudSolrClient;
-import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.cloud.overseer.OverseerAction;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkNodeProps;
-import org.apache.solr.common.params.CollectionParams.CollectionAction;
-import org.apache.solr.common.params.MapSolrParams;
-import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.data.Stat;
+import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
 import static org.apache.solr.cloud.OverseerCollectionConfigSetProcessor.getLeaderNode;
 import static org.apache.solr.cloud.OverseerCollectionConfigSetProcessor.getSortedOverseerNodeNames;
-import static org.apache.solr.cloud.OverseerCollectionMessageHandler.NUM_SLICES;
-import static org.apache.solr.common.util.Utils.makeMap;
-import static org.apache.solr.common.cloud.ZkStateReader.MAX_SHARDS_PER_NODE;
-import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
+import static org.hamcrest.CoreMatchers.not;
 
-@LuceneTestCase.Slow
-@SuppressSSL(bugUrl = "SOLR-5776")
-public class OverseerRolesTest  extends AbstractFullDistribZkTestBase{
+public class OverseerRolesTest extends SolrCloudTestCase {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  protected String getSolrXml() {
-    return "solr.xml";
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(4)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
   }
 
-  public OverseerRolesTest() {
-    sliceCount = 2;
-    fixShardCount(TEST_NIGHTLY ? 6 : 2);
-  }
-
-  @Test
-  public void test() throws Exception {
-    try (CloudSolrClient client = createCloudClient(null))  {
-      testQuitCommand(client);
-      testOverseerRole(client);
+  @Before
+  public void clearAllOverseerRoles() throws Exception {
+    for (String node : OverseerCollectionConfigSetProcessor.getSortedOverseerNodeNames(zkClient())) {
+      CollectionAdminRequest.removeRole(node, "overseer").process(cluster.getSolrClient());
     }
   }
 
-  private void testQuitCommand(CloudSolrClient client) throws Exception{
-    String collectionName = "testOverseerQuit";
-
-    createCollection(collectionName, client);
-
-    waitForRecoveriesToFinish(collectionName, false);
+  @Test
+  public void testQuitCommand() throws Exception {
 
-    SolrZkClient zk = client.getZkStateReader().getZkClient();
-    byte[] data = new byte[0];
-    data = zk.getData("/overseer_elect/leader", null, new Stat(), true);
+    SolrZkClient zk = zkClient();
+    byte[] data = zk.getData("/overseer_elect/leader", null, new Stat(), true);
     Map m = (Map) Utils.fromJSON(data);
     String s = (String) m.get("id");
     String leader = LeaderElector.getNodeName(s);
-    Overseer.getStateUpdateQueue(zk).offer(Utils.toJSON(new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.QUIT.toLower())));
+    log.info("Current overseer: {}", leader);
+    Overseer.getStateUpdateQueue(zk)
+        .offer(Utils.toJSON(new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.QUIT.toLower(),
+                                            "id", s)));
     final TimeOut timeout = new TimeOut(10, TimeUnit.SECONDS);
-    String newLeader=null;
+    String newLeader = null;
     for(;! timeout.hasTimedOut();){
       newLeader = OverseerCollectionConfigSetProcessor.getLeaderNode(zk);
-      if(newLeader!=null && !newLeader.equals(leader)) break;
+      if (newLeader != null && !newLeader.equals(leader))
+        break;
       Thread.sleep(100);
     }
-    assertNotSame( "Leader not changed yet",newLeader,leader);
-
+    assertThat("Leader not changed yet", newLeader, not(leader));
 
-
-    assertTrue("The old leader should have rejoined election ", OverseerCollectionConfigSetProcessor.getSortedOverseerNodeNames(zk).contains(leader));
+    assertTrue("The old leader should have rejoined election",
+        OverseerCollectionConfigSetProcessor.getSortedOverseerNodeNames(zk).contains(leader));
   }
 
+  @Test
+  public void testOverseerRole() throws Exception {
 
-
-
-  private void testOverseerRole(CloudSolrClient client) throws Exception {
-    String collectionName = "testOverseerCol";
-
-    createCollection(collectionName, client);
-
-    waitForRecoveriesToFinish(collectionName, false);
-    List<String> l = OverseerCollectionConfigSetProcessor.getSortedOverseerNodeNames(client.getZkStateReader().getZkClient()) ;
+    List<String> l = OverseerCollectionConfigSetProcessor.getSortedOverseerNodeNames(zkClient()) ;
 
     log.info("All nodes {}", l);
-    String currentLeader = OverseerCollectionConfigSetProcessor.getLeaderNode(client.getZkStateReader().getZkClient());
+    String currentLeader = OverseerCollectionConfigSetProcessor.getLeaderNode(zkClient());
     log.info("Current leader {} ", currentLeader);
     l.remove(currentLeader);
 
     Collections.shuffle(l, random());
     String overseerDesignate = l.get(0);
-    log.info("overseerDesignate {}",overseerDesignate);
-    setOverseerRole(client, CollectionAction.ADDROLE,overseerDesignate);
+    log.info("overseerDesignate {}", overseerDesignate);
+
+    CollectionAdminRequest.addRole(overseerDesignate, "overseer").process(cluster.getSolrClient());
 
     TimeOut timeout = new TimeOut(15, TimeUnit.SECONDS);
 
     boolean leaderchanged = false;
-    for(;!timeout.hasTimedOut();){
-      if(overseerDesignate.equals(OverseerCollectionConfigSetProcessor.getLeaderNode(client.getZkStateReader().getZkClient()))){
+    for (;!timeout.hasTimedOut();) {
+      if (overseerDesignate.equals(OverseerCollectionConfigSetProcessor.getLeaderNode(zkClient()))) {
         log.info("overseer designate is the new overseer");
         leaderchanged =true;
         break;
@@ -136,36 +113,29 @@ public class OverseerRolesTest  extends AbstractFullDistribZkTestBase{
     }
     assertTrue("could not set the new overseer . expected "+
         overseerDesignate + " current order : " +
-        getSortedOverseerNodeNames(client.getZkStateReader().getZkClient()) +
-        " ldr :"+ OverseerCollectionConfigSetProcessor.getLeaderNode(client.getZkStateReader().getZkClient()) ,leaderchanged);
-
-
+        getSortedOverseerNodeNames(zkClient()) +
+        " ldr :"+ OverseerCollectionConfigSetProcessor.getLeaderNode(zkClient()) ,leaderchanged);
 
     //add another node as overseer
-
-
     l.remove(overseerDesignate);
-
     Collections.shuffle(l, random());
 
     String anotherOverseer = l.get(0);
     log.info("Adding another overseer designate {}", anotherOverseer);
-    setOverseerRole(client, CollectionAction.ADDROLE, anotherOverseer);
+    CollectionAdminRequest.addRole(anotherOverseer, "overseer").process(cluster.getSolrClient());
 
-    String currentOverseer = getLeaderNode(client.getZkStateReader().getZkClient());
+    String currentOverseer = getLeaderNode(zkClient());
 
     log.info("Current Overseer {}", currentOverseer);
 
-    String hostPort = currentOverseer.substring(0,currentOverseer.indexOf('_'));
+    String hostPort = currentOverseer.substring(0, currentOverseer.indexOf('_'));
 
     StringBuilder sb = new StringBuilder();
-//
-//
     log.info("hostPort : {}", hostPort);
 
     JettySolrRunner leaderJetty = null;
 
-    for (JettySolrRunner jetty : jettys) {
+    for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
       String s = jetty.getBaseUrl().toString();
       log.info("jetTy {}",s);
       sb.append(s).append(" , ");
@@ -178,49 +148,20 @@ public class OverseerRolesTest  extends AbstractFullDistribZkTestBase{
     assertNotNull("Could not find a jetty2 kill",  leaderJetty);
 
     log.info("leader node {}", leaderJetty.getBaseUrl());
-    log.info ("current election Queue",
-        OverseerCollectionConfigSetProcessor.getSortedElectionNodes(client.getZkStateReader().getZkClient(),
-            "/overseer_elect/election"));
+    log.info("current election Queue",
+        OverseerCollectionConfigSetProcessor.getSortedElectionNodes(zkClient(), "/overseer_elect/election"));
     ChaosMonkey.stop(leaderJetty);
     timeout = new TimeOut(10, TimeUnit.SECONDS);
     leaderchanged = false;
     for (; !timeout.hasTimedOut(); ) {
-      currentOverseer = getLeaderNode(client.getZkStateReader().getZkClient());
+      currentOverseer = getLeaderNode(zkClient());
       if (anotherOverseer.equals(currentOverseer)) {
         leaderchanged = true;
         break;
       }
       Thread.sleep(100);
     }
-    assertTrue("New overseer designate has not become the overseer, expected : " + anotherOverseer + "actual : " + getLeaderNode(client.getZkStateReader().getZkClient()), leaderchanged);
+    assertTrue("New overseer designate has not become the overseer, expected : " + anotherOverseer + "actual : " + getLeaderNode(zkClient()), leaderchanged);
   }
 
-  private void setOverseerRole(CloudSolrClient client, CollectionAction action, String overseerDesignate) throws Exception, IOException {
-    log.info("Adding overseer designate {} ", overseerDesignate);
-    Map m = makeMap(
-        "action", action.toString().toLowerCase(Locale.ROOT),
-        "role", "overseer",
-        "node", overseerDesignate);
-    SolrParams params = new MapSolrParams(m);
-    SolrRequest request = new QueryRequest(params);
-    request.setPath("/admin/collections");
-    client.request(request);
-  }
-
-
-  protected void createCollection(String COLL_NAME, CloudSolrClient client) throws Exception {
-    int replicationFactor = 2;
-    int numShards = 4;
-    int maxShardsPerNode = ((((numShards+1) * replicationFactor) / getCommonCloudSolrClient()
-        .getZkStateReader().getClusterState().getLiveNodes().size())) + 1;
-
-    Map<String, Object> props = makeMap(
-        REPLICATION_FACTOR, replicationFactor,
-        MAX_SHARDS_PER_NODE, maxShardsPerNode,
-        NUM_SLICES, numShards);
-    Map<String,List<Integer>> collectionInfos = new HashMap<>();
-    createCollection(collectionInfos, COLL_NAME, props, client);
-  }
-
-
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12aff1cf/solr/core/src/test/org/apache/solr/cloud/OverseerStatusTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/OverseerStatusTest.java b/solr/core/src/test/org/apache/solr/cloud/OverseerStatusTest.java
index b1899da..80fd38e 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OverseerStatusTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OverseerStatusTest.java
@@ -17,74 +17,56 @@
 package org.apache.solr.cloud;
 
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.client.solrj.response.CollectionAdminResponse;
 import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
-public class OverseerStatusTest extends BasicDistributedZkTest {
+public class OverseerStatusTest extends SolrCloudTestCase {
 
-  public OverseerStatusTest() {
-    schemaString = "schema15.xml";      // we need a string id
-    sliceCount = 1;
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(2)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();;
   }
 
   @Test
-  @ShardsFixed(num = 1)
   public void test() throws Exception {
 
-    waitForThingsToLevelOut(15);
-
     // find existing command counts because collection may be created by base test class too
     int numCollectionCreates = 0, numOverseerCreates = 0;
-    NamedList<Object> resp = new CollectionAdminRequest.OverseerStatus().process(cloudClient).getResponse();
-    if (resp != null) {
-      NamedList<Object> collection_operations = (NamedList<Object>) resp.get("collection_operations");
-      if (collection_operations != null)  {
-        SimpleOrderedMap<Object> createcollection = (SimpleOrderedMap<Object>) collection_operations.get(CollectionParams.CollectionAction.CREATE.toLower());
-        if (createcollection != null && createcollection.get("requests") != null) {
-          numCollectionCreates = (Integer) createcollection.get("requests");
-        }
-        NamedList<Object> overseer_operations = (NamedList<Object>) resp.get("overseer_operations");
-        if (overseer_operations != null)  {
-          createcollection = (SimpleOrderedMap<Object>) overseer_operations.get(CollectionParams.CollectionAction.CREATE.toLower());
-          if (createcollection != null && createcollection.get("requests") != null) {
-            numOverseerCreates = (Integer) createcollection.get("requests");
-          }
-        }
-      }
-    }
 
     String collectionName = "overseer_status_test";
-    CollectionAdminResponse response = createCollection(collectionName, 1, 1, 1);
-    resp = new CollectionAdminRequest.OverseerStatus().process(cloudClient).getResponse();
+    CollectionAdminRequest.createCollection(collectionName, "conf", 1, 1).process(cluster.getSolrClient());
+
+    NamedList<Object> resp = new CollectionAdminRequest.OverseerStatus().process(cluster.getSolrClient()).getResponse();
     NamedList<Object> collection_operations = (NamedList<Object>) resp.get("collection_operations");
     NamedList<Object> overseer_operations = (NamedList<Object>) resp.get("overseer_operations");
-    SimpleOrderedMap<Object> createcollection = (SimpleOrderedMap<Object>) collection_operations.get(CollectionParams.CollectionAction.CREATE.toLower());
+    SimpleOrderedMap<Object> createcollection
+        = (SimpleOrderedMap<Object>) collection_operations.get(CollectionParams.CollectionAction.CREATE.toLower());
     assertEquals("No stats for create in OverseerCollectionProcessor", numCollectionCreates + 1, createcollection.get("requests"));
     createcollection = (SimpleOrderedMap<Object>) overseer_operations.get(CollectionParams.CollectionAction.CREATE.toLower());
     assertEquals("No stats for create in Overseer", numOverseerCreates + 1, createcollection.get("requests"));
 
     // Reload the collection
-    new CollectionAdminRequest.Reload().setCollectionName(collectionName).process(cloudClient);
-
+    CollectionAdminRequest.reloadCollection(collectionName).process(cluster.getSolrClient());
 
-    resp = new CollectionAdminRequest.OverseerStatus().process(cloudClient).getResponse();
+    resp = new CollectionAdminRequest.OverseerStatus().process(cluster.getSolrClient()).getResponse();
     collection_operations = (NamedList<Object>) resp.get("collection_operations");
     SimpleOrderedMap<Object> reload = (SimpleOrderedMap<Object>) collection_operations.get(CollectionParams.CollectionAction.RELOAD.toLower());
     assertEquals("No stats for reload in OverseerCollectionProcessor", 1, reload.get("requests"));
 
     try {
-      new CollectionAdminRequest.SplitShard()
-              .setCollectionName("non_existent_collection")
-              .setShardName("non_existent_shard")
-              .process(cloudClient);
+      CollectionAdminRequest.splitShard("non_existent_collection")
+          .setShardName("non_existent_shard")
+          .process(cluster.getSolrClient());
       fail("Split shard for non existent collection should have failed");
     } catch (Exception e) {
       // expected because we did not correctly specify required params for split
     }
-    resp = new CollectionAdminRequest.OverseerStatus().process(cloudClient).getResponse();
+    resp = new CollectionAdminRequest.OverseerStatus().process(cluster.getSolrClient()).getResponse();
     collection_operations = (NamedList<Object>) resp.get("collection_operations");
     SimpleOrderedMap<Object> split = (SimpleOrderedMap<Object>) collection_operations.get(CollectionParams.CollectionAction.SPLITSHARD.toLower());
     assertEquals("No stats for split in OverseerCollectionProcessor", 1, split.get("errors"));
@@ -111,6 +93,5 @@ public class OverseerStatusTest extends BasicDistributedZkTest {
     assertNotNull(updateState.get("errors"));
     assertNotNull(updateState.get("avgTimePerRequest"));
 
-    waitForThingsToLevelOut(15);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12aff1cf/solr/core/src/test/org/apache/solr/cloud/RemoteQueryErrorTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/RemoteQueryErrorTest.java b/solr/core/src/test/org/apache/solr/cloud/RemoteQueryErrorTest.java
index 24f9696..54503bf 100644
--- a/solr/core/src/test/org/apache/solr/cloud/RemoteQueryErrorTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/RemoteQueryErrorTest.java
@@ -16,58 +16,43 @@
  */
 package org.apache.solr.cloud;
 
-import org.apache.lucene.util.LuceneTestCase.Slow;
 import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
-import java.util.ArrayList;
-import java.util.List;
-
 import static org.junit.internal.matchers.StringContains.containsString;
 
 /**
  * Verify that remote (proxied) queries return proper error messages
  */
-@Slow
-public class RemoteQueryErrorTest extends AbstractFullDistribZkTestBase {
+public class RemoteQueryErrorTest extends SolrCloudTestCase {
 
-  public RemoteQueryErrorTest() {
-    super();
-    sliceCount = 1;
-    fixShardCount(random().nextBoolean() ? 3 : 4);
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(3)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
   }
 
+  // TODO add test for CloudSolrClient as well
+
   @Test
   public void test() throws Exception {
-    handle.clear();
-    handle.put("timestamp", SKIPVAL);
-    
-    waitForThingsToLevelOut(15);
 
-    del("*:*");
-    
-    createCollection("collection2", 2, 1, 10);
-    
-    List<Integer> numShardsNumReplicaList = new ArrayList<>(2);
-    numShardsNumReplicaList.add(2);
-    numShardsNumReplicaList.add(1);
-    checkForCollection("collection2", numShardsNumReplicaList, null);
-    waitForRecoveriesToFinish("collection2", true);
+    CollectionAdminRequest.createCollection("collection", "conf", 2, 1).process(cluster.getSolrClient());
 
-    for (SolrClient solrClient : clients) {
-      try {
-        SolrInputDocument emptyDoc = new SolrInputDocument();
-        solrClient.add(emptyDoc);
-        fail("Expected unique key exception");
-      } catch (SolrException ex) {
-        assertThat(ex.getMessage(), containsString("Document is missing mandatory uniqueKey field: id"));
-      } catch(Exception ex) {
-        fail("Expected a SolrException to occur, instead received: " + ex.getClass());
-      } finally {
-        solrClient.close();
+    for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
+      try (SolrClient client = jetty.newClient()) {
+        SolrException e = expectThrows(SolrException.class, () -> {
+          client.add("collection", new SolrInputDocument());
+        });
+        assertThat(e.getMessage(), containsString("Document is missing mandatory uniqueKey field: id"));
       }
     }
+
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12aff1cf/solr/core/src/test/org/apache/solr/cloud/TestDownShardTolerantSearch.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestDownShardTolerantSearch.java b/solr/core/src/test/org/apache/solr/cloud/TestDownShardTolerantSearch.java
index 01c4440..415d4e4 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestDownShardTolerantSearch.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestDownShardTolerantSearch.java
@@ -16,13 +16,15 @@
  */
 package org.apache.solr.cloud;
 
-import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.common.params.ShardParams;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -34,41 +36,47 @@ import static org.hamcrest.CoreMatchers.is;
  * and also asserts that a meaningful exception is thrown when shards.tolerant=false
  * See SOLR-7566
  */
-public class TestDownShardTolerantSearch extends AbstractFullDistribZkTestBase {
+public class TestDownShardTolerantSearch extends SolrCloudTestCase {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  public TestDownShardTolerantSearch() {
-    sliceCount = 2;
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(2)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
   }
 
   @Test
-  @ShardsFixed(num = 2)
   public void searchingShouldFailWithoutTolerantSearchSetToTrue() throws Exception {
-    waitForRecoveriesToFinish(true);
 
-    indexAbunchOfDocs();
-    commit();
-    QueryResponse response = cloudClient.query(new SolrQuery("*:*").setRows(1));
+    CollectionAdminRequest.createCollection("tolerant", "conf", 2, 1)
+        .process(cluster.getSolrClient());
+
+    UpdateRequest update = new UpdateRequest();
+    for (int i = 0; i < 100; i++) {
+      update.add("id", Integer.toString(i));
+    }
+    update.commit(cluster.getSolrClient(), "tolerant");
+
+    QueryResponse response = cluster.getSolrClient().query("tolerant", new SolrQuery("*:*").setRows(1));
     assertThat(response.getStatus(), is(0));
-    assertThat(response.getResults().getNumFound(), is(66L));
+    assertThat(response.getResults().getNumFound(), is(100L));
 
-    ChaosMonkey.kill(shardToJetty.get(SHARD1).get(0));
+    cluster.stopJettySolrRunner(0);
 
-    response = cloudClient.query(new SolrQuery("*:*").setRows(1).setParam(ShardParams.SHARDS_TOLERANT, true));
+    response = cluster.getSolrClient().query("tolerant", new SolrQuery("*:*").setRows(1).setParam(ShardParams.SHARDS_TOLERANT, true));
     assertThat(response.getStatus(), is(0));
     assertTrue(response.getResults().getNumFound() > 0);
 
     try {
-      cloudClient.query(new SolrQuery("*:*").setRows(1).setParam(ShardParams.SHARDS_TOLERANT, false));
+      cluster.getSolrClient().query("tolerant", new SolrQuery("*:*").setRows(1).setParam(ShardParams.SHARDS_TOLERANT, false));
       fail("Request should have failed because we killed shard1 jetty");
     } catch (SolrServerException e) {
       log.info("error from server", e);
       assertNotNull(e.getCause());
       assertTrue("Error message from server should have the name of the down shard",
-          e.getCause().getMessage().contains(SHARD1));
-    } catch (IOException e) {
-      e.printStackTrace();
+          e.getCause().getMessage().contains("shard"));
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12aff1cf/solr/core/src/test/org/apache/solr/cloud/TestExclusionRuleCollectionAccess.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestExclusionRuleCollectionAccess.java b/solr/core/src/test/org/apache/solr/cloud/TestExclusionRuleCollectionAccess.java
index 9ef2dcd..5bf77c1 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestExclusionRuleCollectionAccess.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestExclusionRuleCollectionAccess.java
@@ -16,34 +16,32 @@
  */
 package org.apache.solr.cloud;
 
-import org.apache.lucene.util.LuceneTestCase;
-import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
-@LuceneTestCase.Slow
-public class TestExclusionRuleCollectionAccess extends AbstractFullDistribZkTestBase {
+public class TestExclusionRuleCollectionAccess extends SolrCloudTestCase {
 
-  public TestExclusionRuleCollectionAccess() {
-    schemaString = "schema15.xml";      // we need a string id
-    sliceCount = 1;
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(1)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
   }
 
   @Test
   public void doTest() throws Exception {
-    CollectionAdminRequest.Create req = new CollectionAdminRequest.Create();
-    req.setCollectionName("css33");
-    req.setNumShards(1);
-    req.process(cloudClient);
-    
-    waitForRecoveriesToFinish("css33", false);
-    
-    try (SolrClient c = createCloudClient("css33")) {
-      c.add(getDoc("id", "1"));
-      c.commit();
-
-      assertEquals("Should have returned 1 result", 1, c.query(params("q", "*:*", "collection", "css33")).getResults().getNumFound());
-    }
+
+    CollectionAdminRequest.createCollection("css33", "conf", 1, 1).process(cluster.getSolrClient());
+
+    new UpdateRequest()
+        .add("id", "1")
+        .commit(cluster.getSolrClient(), "css33");
+
+    assertEquals("Should have returned 1 result", 1,
+        cluster.getSolrClient().query("css33", params("q", "*:*", "collection", "css33")).getResults().getNumFound());
+
   }
   
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12aff1cf/solr/core/src/test/org/apache/solr/security/PKIAuthenticationIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/PKIAuthenticationIntegrationTest.java b/solr/core/src/test/org/apache/solr/security/PKIAuthenticationIntegrationTest.java
index 30fe933..bc4f4e5 100644
--- a/solr/core/src/test/org/apache/solr/security/PKIAuthenticationIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/security/PKIAuthenticationIntegrationTest.java
@@ -18,19 +18,21 @@ package org.apache.solr.security;
 
 import javax.servlet.ServletRequest;
 import javax.servlet.http.HttpServletRequest;
-
 import java.lang.invoke.MethodHandles;
 import java.security.Principal;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.Predicate;
 
-import org.apache.solr.SolrTestCaseJ4;
+import org.apache.http.client.HttpClient;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.QueryRequest;
-import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
+import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.Utils;
+import org.junit.After;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -39,27 +41,32 @@ import static java.util.Collections.singletonMap;
 import static org.apache.solr.common.util.Utils.makeMap;
 import static org.apache.solr.security.TestAuthorizationFramework.verifySecurityStatus;
 
-@SolrTestCaseJ4.SuppressSSL
-public class PKIAuthenticationIntegrationTest extends AbstractFullDistribZkTestBase {
+public class PKIAuthenticationIntegrationTest extends SolrCloudTestCase {
+
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  static final int TIMEOUT = 10000;
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(2)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
+  }
 
   @Test
   public void testPkiAuth() throws Exception {
-    waitForThingsToLevelOut(10);
 
+    CollectionAdminRequest.createCollection("collection", "conf", 2, 1).process(cluster.getSolrClient());
+
+    // TODO make a SolrJ helper class for this
     byte[] bytes = Utils.toJSON(makeMap("authorization", singletonMap("class", MockAuthorizationPlugin.class.getName()),
         "authentication", singletonMap("class", MockAuthenticationPlugin.class.getName())));
+    zkClient().setData(ZkStateReader.SOLR_SECURITY_CONF_PATH, bytes, true);
 
-    try (ZkStateReader zkStateReader = new ZkStateReader(zkServer.getZkAddress(),
-        TIMEOUT, TIMEOUT)) {
-      zkStateReader.getZkClient().setData(ZkStateReader.SOLR_SECURITY_CONF_PATH, bytes, true);
-    }
-    for (JettySolrRunner jetty : jettys) {
+    HttpClient httpClient = cluster.getSolrClient().getHttpClient();
+    for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
       String baseUrl = jetty.getBaseUrl().toString();
-      verifySecurityStatus(cloudClient.getLbClient().getHttpClient(), baseUrl + "/admin/authorization", "authorization/class", MockAuthorizationPlugin.class.getName(), 20);
-      verifySecurityStatus(cloudClient.getLbClient().getHttpClient(), baseUrl + "/admin/authentication", "authentication.enabled", "true", 20);
+      verifySecurityStatus(httpClient, baseUrl + "/admin/authorization", "authorization/class", MockAuthorizationPlugin.class.getName(), 20);
+      verifySecurityStatus(httpClient, baseUrl + "/admin/authentication", "authentication.enabled", "true", 20);
     }
     log.info("Starting test");
     ModifiableSolrParams params = new ModifiableSolrParams();
@@ -95,13 +102,12 @@ public class PKIAuthenticationIntegrationTest extends AbstractFullDistribZkTestB
       }
     };
     QueryRequest query = new QueryRequest(params);
-    query.process(cloudClient);
+    query.process(cluster.getSolrClient(), "collection");
     assertTrue("all nodes must get the user solr , no:of nodes got solr : " + count.get(),count.get() > 2);
   }
 
-  @Override
+  @After
   public void distribTearDown() throws Exception {
-    super.distribTearDown();
     MockAuthenticationPlugin.predicate = null;
     MockAuthorizationPlugin.predicate = null;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12aff1cf/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
index 0beaa55..94750c0a 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
@@ -280,6 +280,8 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
 
     public CollectionAdminRoleRequest(CollectionAction action, String node, String role) {
       super(action);
+      this.node = node;
+      this.role = role;
     }
 
     @Override


[07/50] lucene-solr:jira/solr-8593: SOLR-9877: Null check for metric registry before attempting to use it

Posted by kr...@apache.org.
SOLR-9877: Null check for metric registry before attempting to use it


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

Branch: refs/heads/jira/solr-8593
Commit: 662be93ed11abebaff1d13711f3bffca478ba61e
Parents: 20362de
Author: Shalin Shekhar Mangar <sh...@apache.org>
Authored: Thu Dec 29 09:57:03 2016 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Thu Dec 29 09:57:03 2016 +0530

----------------------------------------------------------------------
 .../solr/util/stats/InstrumentedHttpRequestExecutor.java    | 9 +++++++--
 1 file changed, 7 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/662be93e/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpRequestExecutor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpRequestExecutor.java b/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpRequestExecutor.java
index ad76d73..0426780 100644
--- a/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpRequestExecutor.java
+++ b/solr/core/src/java/org/apache/solr/util/stats/InstrumentedHttpRequestExecutor.java
@@ -53,11 +53,16 @@ public class InstrumentedHttpRequestExecutor extends HttpRequestExecutor impleme
 
   @Override
   public HttpResponse execute(HttpRequest request, HttpClientConnection conn, HttpContext context) throws IOException, HttpException {
-    final Timer.Context timerContext = timer(request).time();
+    Timer.Context timerContext = null;
+    if (metricsRegistry != null)  {
+      timerContext = timer(request).time();
+    }
     try {
       return super.execute(request, conn, context);
     } finally {
-      timerContext.stop();
+      if (timerContext != null) {
+        timerContext.stop();
+      }
     }
   }
 


[05/50] lucene-solr:jira/solr-8593: LUCENE-7595: Improve RAMUsageTester in test-framework to estimate memory usage of runtime classes and work with Java 9 EA (b148+). Disable static field heap usage checker in LuceneTestCase

Posted by kr...@apache.org.
LUCENE-7595: Improve RAMUsageTester in test-framework to estimate memory usage of runtime classes and work with Java 9 EA (b148+). Disable static field heap usage checker in LuceneTestCase


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

Branch: refs/heads/jira/solr-8593
Commit: f29d2b5668296dfcdb8d650305449674faa29847
Parents: 262049f
Author: Uwe Schindler <us...@apache.org>
Authored: Thu Dec 29 01:56:23 2016 +0100
Committer: Uwe Schindler <us...@apache.org>
Committed: Thu Dec 29 01:56:23 2016 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   4 +
 .../apache/lucene/search/TestLRUQueryCache.java |   3 +-
 .../org/apache/lucene/util/LuceneTestCase.java  |  90 ++++++++--------
 .../org/apache/lucene/util/RamUsageTester.java  | 102 ++++++++++++++++---
 4 files changed, 139 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f29d2b56/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 255867d..7a118f1 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -208,6 +208,10 @@ Other
 * LUCENE-7599: Simplify TestRandomChains using Java's built-in Predicate and
   Function interfaces. (Ahmet Arslan via Adrien Grand)
 
+* LUCENE-7595: Improve RAMUsageTester in test-framework to estimate memory usage of
+  runtime classes and work with Java 9 EA (b148+). Disable static field heap usage
+  checker in LuceneTestCase.  (Uwe Schindler, Dawid Weiss)
+
 Build
 
 * LUCENE-7387: fix defaultCodec in build.xml to account for the line ending (hossman)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f29d2b56/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
index 87382f9..9ebacf7 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
@@ -265,6 +265,8 @@ public class TestLRUQueryCache extends LuceneTestCase {
   // This test makes sure that by making the same assumptions as LRUQueryCache, RAMUsageTester
   // computes the same memory usage.
   public void testRamBytesUsedAgreesWithRamUsageTester() throws IOException {
+    assumeFalse("LUCENE-7595: RamUsageTester does not work exact in Java 9 (estimations for maps and lists)", Constants.JRE_IS_MINIMUM_JAVA9);
+    
     final LRUQueryCache queryCache = new LRUQueryCache(1 + random().nextInt(5), 1 + random().nextInt(10000), context -> random().nextBoolean());
     // an accumulator that only sums up memory usage of referenced filters and doc id sets
     final RamUsageTester.Accumulator acc = new RamUsageTester.Accumulator() {
@@ -379,7 +381,6 @@ public class TestLRUQueryCache extends LuceneTestCase {
   // by the cache itself, not cache entries, and we want to make sure that
   // memory usage is not grossly underestimated.
   public void testRamBytesUsedConstantEntryOverhead() throws IOException {
-    LuceneTestCase.assumeFalse("RamUsageTester does not fully work on Java 9", Constants.JRE_IS_MINIMUM_JAVA9);
     final LRUQueryCache queryCache = new LRUQueryCache(1000000, 10000000, context -> true);
 
     final RamUsageTester.Accumulator acc = new RamUsageTester.Accumulator() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f29d2b56/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
index 1848c4e..50139a0 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
@@ -598,51 +598,55 @@ public abstract class LuceneTestCase extends Assert {
    * other.
    */
   @ClassRule
-  public static TestRule classRules = RuleChain
-    .outerRule(new TestRuleIgnoreTestSuites())
-    .around(ignoreAfterMaxFailures)
-    .around(suiteFailureMarker = new TestRuleMarkFailure())
-    .around(new TestRuleAssertionsRequired())
-    .around(new TestRuleLimitSysouts(suiteFailureMarker))
-    .around(tempFilesCleanupRule = new TestRuleTemporaryFilesCleanup(suiteFailureMarker))
-    .around(new StaticFieldsInvariantRule(STATIC_LEAK_THRESHOLD, true) {
-      @Override
-      protected boolean accept(java.lang.reflect.Field field) {
-        // Don't count known classes that consume memory once.
-        if (STATIC_LEAK_IGNORED_TYPES.contains(field.getType().getName())) {
-          return false;
+  public static TestRule classRules;
+  static {
+    RuleChain r = RuleChain.outerRule(new TestRuleIgnoreTestSuites())
+      .around(ignoreAfterMaxFailures)
+      .around(suiteFailureMarker = new TestRuleMarkFailure())
+      .around(new TestRuleAssertionsRequired())
+      .around(new TestRuleLimitSysouts(suiteFailureMarker))
+      .around(tempFilesCleanupRule = new TestRuleTemporaryFilesCleanup(suiteFailureMarker));
+    // TODO LUCENE-7595: Java 9 does not allow to look into runtime classes, so we have to fix the RAM usage checker!
+    if (!Constants.JRE_IS_MINIMUM_JAVA9) {
+      r = r.around(new StaticFieldsInvariantRule(STATIC_LEAK_THRESHOLD, true) {
+        @Override
+        protected boolean accept(java.lang.reflect.Field field) {
+          // Don't count known classes that consume memory once.
+          if (STATIC_LEAK_IGNORED_TYPES.contains(field.getType().getName())) {
+            return false;
+          }
+          // Don't count references from ourselves, we're top-level.
+          if (field.getDeclaringClass() == LuceneTestCase.class) {
+            return false;
+          }
+          return super.accept(field);
         }
-        // Don't count references from ourselves, we're top-level.
-        if (field.getDeclaringClass() == LuceneTestCase.class) {
-          return false;
+      });
+    }
+    classRules = r.around(new NoClassHooksShadowingRule())
+      .around(new NoInstanceHooksOverridesRule() {
+        @Override
+        protected boolean verify(Method key) {
+          String name = key.getName();
+          return !(name.equals("setUp") || name.equals("tearDown"));
         }
-        return super.accept(field);
-      }
-    })
-    .around(new NoClassHooksShadowingRule())
-    .around(new NoInstanceHooksOverridesRule() {
-      @Override
-      protected boolean verify(Method key) {
-        String name = key.getName();
-        return !(name.equals("setUp") || name.equals("tearDown"));
-      }
-    })
-    .around(classNameRule = new TestRuleStoreClassName())
-    .around(new TestRuleRestoreSystemProperties(
-        // Enlist all properties to which we have write access (security manager);
-        // these should be restored to previous state, no matter what the outcome of the test.
-
-        // We reset the default locale and timezone; these properties change as a side-effect
-        "user.language",
-        "user.timezone",
-        
-        // TODO: these should, ideally, be moved to Solr's base class.
-        "solr.directoryFactory",
-        "solr.solr.home",
-        "solr.data.dir"
-        ))
-    .around(classEnvRule = new TestRuleSetupAndRestoreClassEnv());
-
+      })
+      .around(classNameRule = new TestRuleStoreClassName())
+      .around(new TestRuleRestoreSystemProperties(
+          // Enlist all properties to which we have write access (security manager);
+          // these should be restored to previous state, no matter what the outcome of the test.
+  
+          // We reset the default locale and timezone; these properties change as a side-effect
+          "user.language",
+          "user.timezone",
+          
+          // TODO: these should, ideally, be moved to Solr's base class.
+          "solr.directoryFactory",
+          "solr.solr.home",
+          "solr.data.dir"
+          ))
+      .around(classEnvRule = new TestRuleSetupAndRestoreClassEnv());
+  }
 
   // -----------------------------------------------------------------
   // Test level rules.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f29d2b56/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java b/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java
index 9850526..daf81a9 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java
@@ -16,9 +16,12 @@
  */
 package org.apache.lucene.util;
 
+import java.io.ByteArrayOutputStream;
+import java.io.File;
 import java.lang.reflect.Array;
 import java.lang.reflect.Field;
 import java.lang.reflect.Modifier;
+import java.nio.file.Path;
 import java.security.AccessController;
 import java.security.PrivilegedAction;
 import java.util.AbstractList;
@@ -30,6 +33,10 @@ import java.util.IdentityHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.function.ToLongFunction;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
 
 /** Crawls object graph to collect RAM usage for testing */
 public final class RamUsageTester {
@@ -40,9 +47,7 @@ public final class RamUsageTester {
     /** Accumulate transitive references for the provided fields of the given
      *  object into <code>queue</code> and return the shallow size of this object. */
     public long accumulateObject(Object o, long shallowSize, Map<Field, Object> fieldValues, Collection<Object> queue) {
-      for (Object value : fieldValues.values()) {
-        queue.add(value);
-      }
+      queue.addAll(fieldValues.values());
       return shallowSize;
     }
 
@@ -130,10 +135,10 @@ public final class RamUsageTester {
             @Override
             public int size() {
               return len;
-              }
-              
-            };         
-          }
+            }
+            
+          };
+        }
         totalSize += accumulator.accumulateArray(ob, shallowSize, values, stack);
       } else {
         /*
@@ -145,13 +150,36 @@ public final class RamUsageTester {
           if (cachedInfo == null) {
             classCache.put(obClazz, cachedInfo = createCacheEntry(obClazz));
           }
-
-          Map<Field, Object> fieldValues = new HashMap<>();
-          for (Field f : cachedInfo.referenceFields) {
-            fieldValues.put(f, f.get(ob));
+          
+          boolean needsReflection = true;
+          if (Constants.JRE_IS_MINIMUM_JAVA9) {
+            // Java 9: Best guess for some known types, as we cannot precisely look into runtime classes:
+            final ToLongFunction<Object> func = SIMPLE_TYPES.get(obClazz);
+            if (func != null) { // some simple type like String where the size is easy to get from public properties
+              totalSize += accumulator.accumulateObject(ob, cachedInfo.alignedShallowInstanceSize + func.applyAsLong(ob), 
+                  Collections.emptyMap(), stack);
+              needsReflection = false;
+            } else if (ob instanceof Iterable) {
+              final List<Object> values = StreamSupport.stream(((Iterable<?>) ob).spliterator(), false)
+                  .collect(Collectors.toList());
+              totalSize += accumulator.accumulateArray(ob, cachedInfo.alignedShallowInstanceSize + RamUsageEstimator.NUM_BYTES_ARRAY_HEADER, values, stack);
+              needsReflection = false;
+            }  else if (ob instanceof Map) {
+              final List<Object> values = ((Map<?,?>) ob).entrySet().stream()
+                  .flatMap(e -> Stream.of(e.getKey(), e.getValue()))
+                  .collect(Collectors.toList());
+              totalSize += accumulator.accumulateArray(ob, cachedInfo.alignedShallowInstanceSize + RamUsageEstimator.NUM_BYTES_ARRAY_HEADER, values, stack);
+              totalSize += RamUsageEstimator.NUM_BYTES_ARRAY_HEADER;
+              needsReflection = false;
+            }
+          }
+          if (needsReflection) {
+            final Map<Field, Object> fieldValues = new HashMap<>();
+            for (Field f : cachedInfo.referenceFields) {
+              fieldValues.put(f, f.get(ob));
+            }
+            totalSize += accumulator.accumulateObject(ob, cachedInfo.alignedShallowInstanceSize, fieldValues, stack);
           }
-
-          totalSize += accumulator.accumulateObject(ob, cachedInfo.alignedShallowInstanceSize, fieldValues, stack);
         } catch (IllegalAccessException e) {
           // this should never happen as we enabled setAccessible().
           throw new RuntimeException("Reflective field access failed?", e);
@@ -167,7 +195,41 @@ public final class RamUsageTester {
     return totalSize;
   }
   
-
+  /**
+   * This map contains a function to calculate sizes of some "simple types" like String just from their public properties.
+   * This is needed for Java 9, which does not allow to look into runtime class fields.
+   */
+  @SuppressWarnings("serial")
+  private static final Map<Class<?>, ToLongFunction<Object>> SIMPLE_TYPES = Collections.unmodifiableMap(new IdentityHashMap<Class<?>, ToLongFunction<Object>>() {
+    { init(); }
+    
+    @SuppressForbidden(reason = "We measure some forbidden classes")
+    private void init() {
+      // String types:
+      a(String.class, v -> charArraySize(v.length())); // may not be correct with Java 9's compact strings!
+      a(StringBuilder.class, v -> charArraySize(v.capacity()));
+      a(StringBuffer.class, v -> charArraySize(v.capacity()));
+      // Types with large buffers:
+      a(ByteArrayOutputStream.class, v -> byteArraySize(v.size()));
+      // For File and Path, we just take the length of String representation as approximation:
+      a(File.class, v -> charArraySize(v.toString().length()));
+      a(Path.class, v -> charArraySize(v.toString().length()));
+    }
+    
+    @SuppressWarnings("unchecked")
+    private <T> void a(Class<T> clazz, ToLongFunction<T> func) {
+      put(clazz, (ToLongFunction<Object>) func);
+    }
+    
+    private long charArraySize(int len) {
+      return RamUsageEstimator.alignObjectSize((long)RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + (long)Character.BYTES * len);
+    }
+    
+    private long byteArraySize(int len) {
+      return RamUsageEstimator.alignObjectSize((long)RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + len);
+    }
+  });
+  
   /**
    * Cached information about a given class.   
    */
@@ -202,8 +264,16 @@ public final class RamUsageTester {
             shallowInstanceSize = RamUsageEstimator.adjustForField(shallowInstanceSize, f);
   
             if (!f.getType().isPrimitive()) {
-              f.setAccessible(true);
-              referenceFields.add(f);
+              try {
+                f.setAccessible(true);
+                referenceFields.add(f);
+              } catch (RuntimeException re) {
+                if ("java.lang.reflect.InaccessibleObjectException".equals(re.getClass().getName())) {
+                  // LUCENE-7595: this is Java 9, which prevents access to fields in foreign modules
+                } else {
+                  throw re;
+                }
+              }
             }
           }
         }


[45/50] lucene-solr:jira/solr-8593: don't allow position length < 1

Posted by kr...@apache.org.
don't allow position length < 1


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

Branch: refs/heads/jira/solr-8593
Commit: 7b2e3db5531d42d91c2718737c63c2ce4d873c8e
Parents: 018df31
Author: Mike McCandless <mi...@apache.org>
Authored: Tue Jan 3 05:46:11 2017 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Tue Jan 3 05:46:11 2017 -0500

----------------------------------------------------------------------
 .../analysis/tokenattributes/PackedTokenAttributeImpl.java  | 9 ++++++---
 1 file changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7b2e3db5/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PackedTokenAttributeImpl.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PackedTokenAttributeImpl.java b/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PackedTokenAttributeImpl.java
index aaa3316..c89a374 100644
--- a/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PackedTokenAttributeImpl.java
+++ b/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PackedTokenAttributeImpl.java
@@ -46,9 +46,9 @@ public class PackedTokenAttributeImpl extends CharTermAttributeImpl
    */
   @Override
   public void setPositionIncrement(int positionIncrement) {
-    if (positionIncrement < 0)
-      throw new IllegalArgumentException
-        ("Increment must be zero or greater: " + positionIncrement);
+    if (positionIncrement < 0) {
+      throw new IllegalArgumentException("Increment must be zero or greater: " + positionIncrement);
+    }
     this.positionIncrement = positionIncrement;
   }
 
@@ -67,6 +67,9 @@ public class PackedTokenAttributeImpl extends CharTermAttributeImpl
    */
   @Override
   public void setPositionLength(int positionLength) {
+    if (positionLength < 1) {
+      throw new IllegalArgumentException("Position length must be 1 or greater: got " + positionLength);
+    }
     this.positionLength = positionLength;
   }
 


[23/50] lucene-solr:jira/solr-8593: LUCENE-7564: add missing javadocs

Posted by kr...@apache.org.
LUCENE-7564: add missing javadocs


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

Branch: refs/heads/jira/solr-8593
Commit: 6b00ee5175d55d2f2a25ce6539dc12277022c898
Parents: 93fdc20
Author: Steve Rowe <sa...@apache.org>
Authored: Thu Dec 29 16:08:35 2016 -0500
Committer: Steve Rowe <sa...@apache.org>
Committed: Thu Dec 29 16:08:53 2016 -0500

----------------------------------------------------------------------
 .../lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java    | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6b00ee51/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
index 2fbe4a8..81880d4 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
@@ -137,6 +137,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
   /** {@link IndexSearcher} used for lookups. */
   protected SearcherManager searcherMgr;
   
+  /** Used to manage concurrent access to searcherMgr */
   protected final Object searcherMgrLock = new Object();
 
   /** Default minimum number of leading characters before


[04/50] lucene-solr:jira/solr-8593: SOLR-9899: StandardDirectoryFactory should use optimizations for all FilterDirectorys not just NRTCachingDirectory.

Posted by kr...@apache.org.
SOLR-9899: StandardDirectoryFactory should use optimizations for all FilterDirectorys not just NRTCachingDirectory.


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

Branch: refs/heads/jira/solr-8593
Commit: 262049fc8f60a166f0eed0aef5d7ddd1e7c90bc7
Parents: 96ed221
Author: markrmiller <ma...@apache.org>
Authored: Wed Dec 28 17:42:41 2016 -0500
Committer: markrmiller <ma...@apache.org>
Committed: Wed Dec 28 18:52:20 2016 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                     |  3 +++
 .../java/org/apache/solr/core/DirectoryFactory.java  | 11 +++++++++++
 .../apache/solr/core/StandardDirectoryFactory.java   | 15 ---------------
 3 files changed, 14 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/262049fc/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index c3cac28..501527a 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -343,6 +343,9 @@ Other Changes
 
 * SOLR-9448: providing a test to workaround a differently named uniqueKey field (Mikhail Khludnev)
 
+* SOLR-9899: StandardDirectoryFactory should use optimizations for all FilterDirectorys not just NRTCachingDirectory.
+  (Mark Miller)
+
 ==================  6.3.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/262049fc/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java b/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
index 396a30d..136a0a6 100644
--- a/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
@@ -27,6 +27,7 @@ import java.util.Collections;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FilterDirectory;
 import org.apache.lucene.store.FlushInfo;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.LockFactory;
@@ -371,4 +372,14 @@ public abstract class DirectoryFactory implements NamedListInitializedPlugin,
   public void initCoreContainer(CoreContainer cc) {
     this.coreContainer = cc;
   }
+  
+  // special hack to work with FilterDirectory
+  protected Directory getBaseDir(Directory dir) {
+    Directory baseDir = dir;
+    while (baseDir instanceof FilterDirectory) {
+      baseDir = ((FilterDirectory)baseDir).getDelegate();
+    } 
+    
+    return baseDir;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/262049fc/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java b/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java
index d418137..b24be14 100644
--- a/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java
@@ -30,7 +30,6 @@ import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.LockFactory;
-import org.apache.lucene.store.NRTCachingDirectory;
 import org.apache.lucene.store.NativeFSLockFactory;
 import org.apache.lucene.store.NoLockFactory;
 import org.apache.lucene.store.SimpleFSLockFactory;
@@ -116,8 +115,6 @@ public class StandardDirectoryFactory extends CachingDirectoryFactory {
    * carefully - some Directory wrappers will
    * cache files for example.
    * 
-   * This implementation works with NRTCachingDirectory.
-   * 
    * You should first {@link Directory#sync(java.util.Collection)} any file that will be 
    * moved or avoid cached files through settings.
    * 
@@ -144,18 +141,6 @@ public class StandardDirectoryFactory extends CachingDirectoryFactory {
 
     super.move(fromDir, toDir, fileName, ioContext);
   }
-
-  // special hack to work with NRTCachingDirectory
-  private Directory getBaseDir(Directory dir) {
-    Directory baseDir;
-    if (dir instanceof NRTCachingDirectory) {
-      baseDir = ((NRTCachingDirectory)dir).getDelegate();
-    } else {
-      baseDir = dir;
-    }
-    
-    return baseDir;
-  }
   
   // perform an atomic rename if possible
   public void renameWithOverwrite(Directory dir, String fileName, String toName) throws IOException {


[12/50] lucene-solr:jira/solr-8593: tests: bump timeout

Posted by kr...@apache.org.
tests: bump timeout


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

Branch: refs/heads/jira/solr-8593
Commit: b4de6288fb739b53ad138a16bc862130dc9318a8
Parents: 5f55ae0
Author: markrmiller <ma...@apache.org>
Authored: Thu Dec 29 05:59:25 2016 -0500
Committer: markrmiller <ma...@apache.org>
Committed: Thu Dec 29 05:59:25 2016 -0500

----------------------------------------------------------------------
 .../org/apache/solr/cloud/LeaderFailoverAfterPartitionTest.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4de6288/solr/core/src/test/org/apache/solr/cloud/LeaderFailoverAfterPartitionTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/LeaderFailoverAfterPartitionTest.java b/solr/core/src/test/org/apache/solr/cloud/LeaderFailoverAfterPartitionTest.java
index 9f1abde..f172267 100644
--- a/solr/core/src/test/org/apache/solr/cloud/LeaderFailoverAfterPartitionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/LeaderFailoverAfterPartitionTest.java
@@ -159,7 +159,7 @@ public class LeaderFailoverAfterPartitionTest extends HttpPartitionTest {
     
     proxy0.reopen();
     
-    long timeout = System.nanoTime() + TimeUnit.NANOSECONDS.convert(60, TimeUnit.SECONDS);
+    long timeout = System.nanoTime() + TimeUnit.NANOSECONDS.convert(90, TimeUnit.SECONDS);
     while (System.nanoTime() < timeout) {
       List<Replica> activeReps = getActiveOrRecoveringReplicas(testCollectionName, "shard1");
       if (activeReps.size() >= 2) break;


[28/50] lucene-solr:jira/solr-8593: SOLR-9843 Fix up DocValuesNotIndexedTest failures (cherry picked from commit f6a3557)

Posted by kr...@apache.org.
SOLR-9843 Fix up DocValuesNotIndexedTest failures
(cherry picked from commit f6a3557)


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

Branch: refs/heads/jira/solr-8593
Commit: 3ccd15a7658ad2821e8a2d2916781265db6f3afe
Parents: cb266d5
Author: Erick Erickson <er...@apache.org>
Authored: Thu Dec 29 18:10:34 2016 -0800
Committer: Erick Erickson <er...@apache.org>
Committed: Thu Dec 29 18:14:39 2016 -0800

----------------------------------------------------------------------
 .../org/apache/solr/cloud/DocValuesNotIndexedTest.java    | 10 ----------
 1 file changed, 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ccd15a7/solr/core/src/test/org/apache/solr/cloud/DocValuesNotIndexedTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/DocValuesNotIndexedTest.java b/solr/core/src/test/org/apache/solr/cloud/DocValuesNotIndexedTest.java
index f5257f8..be9f9a3 100644
--- a/solr/core/src/test/org/apache/solr/cloud/DocValuesNotIndexedTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/DocValuesNotIndexedTest.java
@@ -256,10 +256,6 @@ public class DocValuesNotIndexedTest extends SolrCloudTestCase {
       solrQuery.addSort("id", SolrQuery.ORDER.asc);
       final QueryResponse rsp = client.query(COLLECTION, solrQuery);
       SolrDocumentList res = rsp.getResults();
-      //TODO remove after SOLR-9843
-      if (order.length != res.getNumFound()) {
-        log.error("(3) About to fail, response is: " + rsp.toString());
-      }
       assertEquals("Should have exactly " + order.length + " documents returned", order.length, res.getNumFound());
       String expected;
       for (int idx = 0; idx < res.size(); ++idx) {
@@ -305,10 +301,6 @@ public class DocValuesNotIndexedTest extends SolrCloudTestCase {
       if (prop.getName().startsWith("bool")) expected = 3; //true, false and null
 
       List<Group> fieldCommandGroups = fieldCommand.getValues();
-      //TODO: remove me since this is excessive in the normal case, this is in for SOLR-9843
-      if (expected != fieldCommandGroups.size()) {
-        log.error("(1) About to fail assert, response is: " + rsp.toString());
-      }
       assertEquals("Did not find the expected number of groups for field " + prop.getName(), expected, fieldCommandGroups.size());
     }
   }
@@ -381,8 +373,6 @@ public class DocValuesNotIndexedTest extends SolrCloudTestCase {
               break;
             
             default:
-              //TODO remove me after SOLR-9843
-              log.error("(2) About to fail, response is: " + rsp.toString());
               fail("Unexpected number of elements in the group for " + prop.getName() + ": " + grp.getResult().size());
           }
         }


[22/50] lucene-solr:jira/solr-8593: LUCENE-7564: Force single-threaded access to the AnalyzingInfixSuggester's SearcherManager when performing an acquire() or reassigning. This fixes failures in AnalyzingInfixSuggester.testRandomNRT().

Posted by kr...@apache.org.
LUCENE-7564: Force single-threaded access to the AnalyzingInfixSuggester's SearcherManager when performing an acquire() or reassigning.  This fixes failures in AnalyzingInfixSuggester.testRandomNRT().


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

Branch: refs/heads/jira/solr-8593
Commit: 93fdc20736d6e13736aceb091ab978bd8e03fcbb
Parents: a4335c0
Author: Steve Rowe <sa...@apache.org>
Authored: Thu Dec 29 15:51:37 2016 -0500
Committer: Steve Rowe <sa...@apache.org>
Committed: Thu Dec 29 15:51:37 2016 -0500

----------------------------------------------------------------------
 .../analyzing/AnalyzingInfixSuggester.java      | 136 +++++++++++--------
 1 file changed, 81 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/93fdc207/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
index b8c2dbd..2fbe4a8 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
@@ -136,6 +136,8 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
 
   /** {@link IndexSearcher} used for lookups. */
   protected SearcherManager searcherMgr;
+  
+  protected final Object searcherMgrLock = new Object();
 
   /** Default minimum number of leading characters before
    *  PrefixQuery is used (4). */
@@ -275,53 +277,55 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
   @Override
   public void build(InputIterator iter) throws IOException {
     
-    if (searcherMgr != null) {
-      searcherMgr.close();
-      searcherMgr = null;
-    }
+    synchronized (searcherMgrLock) {
+      if (searcherMgr != null) {
+        searcherMgr.close();
+        searcherMgr = null;
+      }
 
-    if (writer != null) {
-      writer.close();
-      writer = null;
-    }
+      if (writer != null) {
+        writer.close();
+        writer = null;
+      }
 
-    boolean success = false;
-    try {
-      // First pass: build a temporary normal Lucene index,
-      // just indexing the suggestions as they iterate:
-      writer = new IndexWriter(dir,
-                               getIndexWriterConfig(getGramAnalyzer(), IndexWriterConfig.OpenMode.CREATE));
-      //long t0 = System.nanoTime();
-
-      // TODO: use threads?
-      BytesRef text;
-      while ((text = iter.next()) != null) {
-        BytesRef payload;
-        if (iter.hasPayloads()) {
-          payload = iter.payload();
-        } else {
-          payload = null;
-        }
+      boolean success = false;
+      try {
+        // First pass: build a temporary normal Lucene index,
+        // just indexing the suggestions as they iterate:
+        writer = new IndexWriter(dir,
+            getIndexWriterConfig(getGramAnalyzer(), IndexWriterConfig.OpenMode.CREATE));
+        //long t0 = System.nanoTime();
+
+        // TODO: use threads?
+        BytesRef text;
+        while ((text = iter.next()) != null) {
+          BytesRef payload;
+          if (iter.hasPayloads()) {
+            payload = iter.payload();
+          } else {
+            payload = null;
+          }
 
-        add(text, iter.contexts(), iter.weight(), payload);
-      }
+          add(text, iter.contexts(), iter.weight(), payload);
+        }
 
-      //System.out.println("initial indexing time: " + ((System.nanoTime()-t0)/1000000) + " msec");
-      if (commitOnBuild || closeIndexWriterOnBuild) {
-        commit();
-      }
-      searcherMgr = new SearcherManager(writer, null);
-      success = true;
-    } finally {
-      if (success) {
-        if (closeIndexWriterOnBuild) {
-          writer.close();
-          writer = null;
+        //System.out.println("initial indexing time: " + ((System.nanoTime()-t0)/1000000) + " msec");
+        if (commitOnBuild || closeIndexWriterOnBuild) {
+          commit();
         }
-      } else {  // failure
-        if (writer != null) {
-          writer.rollback();
-          writer = null;
+        searcherMgr = new SearcherManager(writer, null);
+        success = true;
+      } finally {
+        if (success) {
+          if (closeIndexWriterOnBuild) {
+            writer.close();
+            writer = null;
+          }
+        } else {  // failure
+          if (writer != null) {
+            writer.rollback();
+            writer = null;
+          }
         }
       }
     }
@@ -369,10 +373,12 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
       } else {
         writer = new IndexWriter(dir, getIndexWriterConfig(getGramAnalyzer(), IndexWriterConfig.OpenMode.CREATE));
       }
-      SearcherManager oldSearcherMgr = searcherMgr;
-      searcherMgr = new SearcherManager(writer, null);
-      if (oldSearcherMgr != null) {
-        oldSearcherMgr.close();
+      synchronized (searcherMgrLock) {
+        SearcherManager oldSearcherMgr = searcherMgr;
+        searcherMgr = new SearcherManager(writer, null);
+        if (oldSearcherMgr != null) {
+          oldSearcherMgr.close();
+        }
       }
     }
   }
@@ -642,7 +648,12 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
     // only retrieve the first num hits now:
     Collector c2 = new EarlyTerminatingSortingCollector(c, SORT, num);
     List<LookupResult> results = null;
-    IndexSearcher searcher = searcherMgr.acquire();
+    SearcherManager mgr;
+    IndexSearcher searcher;
+    synchronized (searcherMgrLock) {
+      mgr = searcherMgr; // acquire & release on same SearcherManager, via local reference
+      searcher = mgr.acquire();
+    }
     try {
       //System.out.println("got searcher=" + searcher);
       searcher.search(finalQuery, c2);
@@ -653,7 +664,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
       // hits = searcher.search(query, null, num, SORT);
       results = createResults(searcher, hits, num, key, doHighlight, matchedTokens, prefixToken);
     } finally {
-      searcherMgr.release(searcher);
+      mgr.release(searcher);
     }
 
     //System.out.println((System.currentTimeMillis() - t0) + " msec for infix suggest");
@@ -853,7 +864,12 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
     long mem = RamUsageEstimator.shallowSizeOf(this);
     try {
       if (searcherMgr != null) {
-        IndexSearcher searcher = searcherMgr.acquire();
+        SearcherManager mgr;
+        IndexSearcher searcher;
+        synchronized (searcherMgrLock) {
+          mgr = searcherMgr; // acquire & release on same SearcherManager, via local reference
+          searcher = mgr.acquire();
+        }
         try {
           for (LeafReaderContext context : searcher.getIndexReader().leaves()) {
             LeafReader reader = FilterLeafReader.unwrap(context.reader());
@@ -862,7 +878,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
             }
           }
         } finally {
-          searcherMgr.release(searcher);
+          mgr.release(searcher);
         }
       }
       return mem;
@@ -876,7 +892,12 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
     List<Accountable> resources = new ArrayList<>();
     try {
       if (searcherMgr != null) {
-        IndexSearcher searcher = searcherMgr.acquire();
+        SearcherManager mgr;
+        IndexSearcher searcher;
+        synchronized (searcherMgrLock) {
+          mgr = searcherMgr; // acquire & release on same SearcherManager, via local reference
+          searcher = mgr.acquire();
+        }
         try {
           for (LeafReaderContext context : searcher.getIndexReader().leaves()) {
             LeafReader reader = FilterLeafReader.unwrap(context.reader());
@@ -885,7 +906,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
             }
           }
         } finally {
-          searcherMgr.release(searcher);
+          mgr.release(searcher);
         }
       }
       return Collections.unmodifiableList(resources);
@@ -899,11 +920,16 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
     if (searcherMgr == null) {
       return 0;
     }
-    IndexSearcher searcher = searcherMgr.acquire();
+    SearcherManager mgr;
+    IndexSearcher searcher;
+    synchronized (searcherMgrLock) {
+      mgr = searcherMgr; // acquire & release on same SearcherManager, via local reference
+      searcher = mgr.acquire();
+    }
     try {
       return searcher.getIndexReader().numDocs();
     } finally {
-      searcherMgr.release(searcher);
+      mgr.release(searcher);
     }
   }
-};
+}


[06/50] lucene-solr:jira/solr-8593: SOLR-9897: Add hl.requireFieldMatch=false support when using the UnifiedHighlighter

Posted by kr...@apache.org.
SOLR-9897: Add hl.requireFieldMatch=false support when using the UnifiedHighlighter


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

Branch: refs/heads/jira/solr-8593
Commit: 20362deb7e6814c1922163595e7edeb652d3ce37
Parents: f29d2b5
Author: David Smiley <ds...@apache.org>
Authored: Wed Dec 28 22:57:44 2016 -0500
Committer: David Smiley <ds...@apache.org>
Committed: Wed Dec 28 22:57:44 2016 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                     |  7 +++++--
 .../solr/highlight/UnifiedSolrHighlighter.java       | 15 ++++++++++++++-
 .../solr/highlight/TestUnifiedSolrHighlighter.java   | 10 ++++++++++
 .../apache/solr/common/params/HighlightParams.java   |  2 +-
 4 files changed, 30 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/20362deb/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 501527a..852a306 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -109,8 +109,8 @@ Upgrade Notes
 
 * SOLR-9708: You are encouraged to try out the UnifiedHighlighter by setting hl.method=unified and report feedback. It
   might become the default in 7.0. It's more efficient/faster than the other highlighters, especially compared to the
-  original Highlighter. That said, some options aren't supported yet, notably hl.fragsize and
-  hl.requireFieldMatch=false. It will get more features in time, especially with your input. See HighlightParams.java
+  original Highlighter. That said, some options aren't supported yet, notably hl.fragsize.
+  It will get more features in time, especially with your input. See HighlightParams.java
   for a listing of highlight parameters annotated with which highlighters use them.
   hl.useFastVectorHighlighter is now considered deprecated in lieu of hl.method=fastVector.
 
@@ -199,6 +199,9 @@ New Features
 
 * SOLR-9880: Add Ganglia, Graphite and SLF4J metrics reporters. (ab)
 
+* SOLR-9897: Add hl.requireFieldMatch toggle support when using the UnifiedHighlighter.  Defaults to false like the
+  other highlighters that support this. (David Smiley)
+
 Optimizations
 ----------------------
 * SOLR-9704: Facet Module / JSON Facet API: Optimize blockChildren facets that have

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/20362deb/solr/core/src/java/org/apache/solr/highlight/UnifiedSolrHighlighter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/highlight/UnifiedSolrHighlighter.java b/solr/core/src/java/org/apache/solr/highlight/UnifiedSolrHighlighter.java
index c38546e..910fa2b 100644
--- a/solr/core/src/java/org/apache/solr/highlight/UnifiedSolrHighlighter.java
+++ b/solr/core/src/java/org/apache/solr/highlight/UnifiedSolrHighlighter.java
@@ -23,6 +23,7 @@ import java.util.List;
 import java.util.Locale;
 import java.util.Map;
 import java.util.Set;
+import java.util.function.Predicate;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.search.DocIdSetIterator;
@@ -221,9 +222,10 @@ public class UnifiedSolrHighlighter extends SolrHighlighter implements PluginInf
    * From {@link #getHighlighter(org.apache.solr.request.SolrQueryRequest)}.
    */
   protected static class SolrExtendedUnifiedHighlighter extends UnifiedHighlighter {
+    protected final static Predicate<String> NOT_REQUIRED_FIELD_MATCH_PREDICATE = s -> true;
     protected final SolrParams params;
-    protected final IndexSchema schema;
 
+    protected final IndexSchema schema;
     protected final RTimerTree loadFieldValuesTimer;
 
     public SolrExtendedUnifiedHighlighter(SolrQueryRequest req) {
@@ -360,6 +362,17 @@ public class UnifiedSolrHighlighter extends SolrHighlighter implements PluginInf
       return params.getFieldBool(field, HighlightParams.USE_PHRASE_HIGHLIGHTER, true);
     }
 
+    @Override
+    protected Predicate<String> getFieldMatcher(String field) {
+      // TODO define hl.queryFieldPattern as a more advanced alternative to hl.requireFieldMatch.
+
+      // note that the UH & PH at Lucene level default to effectively "true"
+      if (params.getFieldBool(field, HighlightParams.FIELD_MATCH, false)) {
+        return field::equals; // requireFieldMatch
+      } else {
+        return NOT_REQUIRED_FIELD_MATCH_PREDICATE;
+      }
+    }
   }
 
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/20362deb/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java b/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java
index 95754a4..e2511be 100644
--- a/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java
+++ b/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java
@@ -225,5 +225,15 @@ public class TestUnifiedSolrHighlighter extends SolrTestCaseJ4 {
         req("q", "text:document", "sort", "id asc", "hl", "true", "hl.encoder", "html"),
         "//lst[@name='highlighting']/lst[@name='103']/arr[@name='text']/str='<em>Document</em>&#32;one&#32;has&#32;a&#32;first&#32;&lt;i&gt;sentence&lt;&#x2F;i&gt;&#46;'");
   }
+
+  public void testRequireFieldMatch() {
+    // We highlight on field text3 (hl.fl), but our query only references the "text" field. Nonetheless, the query word
+    //  "document" is found in all fields here.
+
+    assertQ(req("q", "id:101", "hl", "true", "hl.q", "text:document", "hl.fl", "text3"), //hl.requireFieldMatch is false by default
+        "count(//lst[@name='highlighting']/lst[@name='101']/arr[@name='text3']/*)=1");
+    assertQ(req("q", "id:101", "hl", "true", "hl.q", "text:document", "hl.fl", "text3", "hl.requireFieldMatch", "true"),
+        "count(//lst[@name='highlighting']/lst[@name='101']/arr[@name='text3']/*)=0");
+  }
   
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/20362deb/solr/solrj/src/java/org/apache/solr/common/params/HighlightParams.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/params/HighlightParams.java b/solr/solrj/src/java/org/apache/solr/common/params/HighlightParams.java
index fd752bf..917e9f5 100644
--- a/solr/solrj/src/java/org/apache/solr/common/params/HighlightParams.java
+++ b/solr/solrj/src/java/org/apache/solr/common/params/HighlightParams.java
@@ -38,7 +38,7 @@ public interface HighlightParams {
   // query interpretation
   public static final String Q           = HIGHLIGHT+".q"; // all
   public static final String QPARSER     = HIGHLIGHT+".qparser"; // all
-  public static final String FIELD_MATCH = HIGHLIGHT+".requireFieldMatch"; // OH, FVH
+  public static final String FIELD_MATCH = HIGHLIGHT+".requireFieldMatch"; // OH, FVH, UH
   public static final String USE_PHRASE_HIGHLIGHTER = HIGHLIGHT+".usePhraseHighlighter"; // OH, FVH, UH
   public static final String HIGHLIGHT_MULTI_TERM = HIGHLIGHT+".highlightMultiTerm"; // all
 


[03/50] lucene-solr:jira/solr-8593: SOLR-9859: replication.properties cannot be updated after being written and neither eplication.properties or ndex.properties are durable in the face of a crash.

Posted by kr...@apache.org.
SOLR-9859: replication.properties cannot be updated after being written and neither eplication.properties or ndex.properties are durable in the face of a crash.


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

Branch: refs/heads/jira/solr-8593
Commit: 96ed221fb6924dd167591004a5eaf70d53f92e4f
Parents: dc6dcdd
Author: markrmiller <ma...@apache.org>
Authored: Wed Dec 28 17:40:03 2016 -0500
Committer: markrmiller <ma...@apache.org>
Committed: Wed Dec 28 18:52:19 2016 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  3 +++
 .../org/apache/solr/core/DirectoryFactory.java  | 15 ++++++++++++
 .../apache/solr/core/HdfsDirectoryFactory.java  |  9 +++++++
 .../solr/core/StandardDirectoryFactory.java     | 22 +++++++++++++++++
 .../org/apache/solr/handler/IndexFetcher.java   | 25 +++++++++++---------
 .../solr/handler/TestReplicationHandler.java    |  7 +++++-
 6 files changed, 69 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/96ed221f/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 7a708a6..c3cac28 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -287,6 +287,9 @@ Bug Fixes
 
 * SOLR-9699,SOLR-4668: fix exception from core status in parallel with core reload (Mikhail Khludnev)
 
+* SOLR-9859: replication.properties cannot be updated after being written and neither replication.properties or 
+  index.properties are durable in the face of a crash. (Pushkar Raste, Chris de Kok, Cao Manh Dat, Mark Miller)
+
 Other Changes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/96ed221f/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java b/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
index 228260a..396a30d 100644
--- a/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
@@ -19,6 +19,7 @@ package org.apache.solr.core;
 import java.io.Closeable;
 import java.io.File;
 import java.io.FileFilter;
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.Collection;
@@ -184,6 +185,20 @@ public abstract class DirectoryFactory implements NamedListInitializedPlugin,
     fromDir.deleteFile(fileName);
   }
   
+  // sub classes perform an atomic rename if possible, otherwise fall back to delete + rename
+  // this is important to support for index roll over durability after crashes
+  public void renameWithOverwrite(Directory dir, String fileName, String toName) throws IOException {
+    try {
+      dir.deleteFile(toName);
+    } catch (FileNotFoundException e) {
+
+    } catch (Exception e) {
+      log.error("Exception deleting file", e);
+    }
+
+    dir.rename(fileName, toName);
+  }
+  
   /**
    * Returns the Directory for a given path, using the specified rawLockType.
    * Will return the same Directory instance for the same path.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/96ed221f/solr/core/src/java/org/apache/solr/core/HdfsDirectoryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/HdfsDirectoryFactory.java b/solr/core/src/java/org/apache/solr/core/HdfsDirectoryFactory.java
index b003287..d481e03 100644
--- a/solr/core/src/java/org/apache/solr/core/HdfsDirectoryFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/HdfsDirectoryFactory.java
@@ -29,8 +29,10 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -568,4 +570,11 @@ public class HdfsDirectoryFactory extends CachingDirectoryFactory implements Sol
       }
     }
   }
+  
+  // perform an atomic rename if possible
+  public void renameWithOverwrite(Directory dir, String fileName, String toName) throws IOException {
+    String hdfsDirPath = getPath(dir);
+    FileContext fileContext = FileContext.getFileContext(getConf());
+    fileContext.rename(new Path(hdfsDirPath + "/" + fileName), new Path(hdfsDirPath + "/" + toName), Options.Rename.OVERWRITE);
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/96ed221f/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java b/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java
index 532655b..d418137 100644
--- a/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java
@@ -18,6 +18,11 @@ package org.apache.solr.core;
 import java.io.File;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
+import java.nio.file.AtomicMoveNotSupportedException;
+import java.nio.file.FileSystems;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardCopyOption;
 import java.util.Locale;
 
 import org.apache.commons.io.FileUtils;
@@ -151,5 +156,22 @@ public class StandardDirectoryFactory extends CachingDirectoryFactory {
     
     return baseDir;
   }
+  
+  // perform an atomic rename if possible
+  public void renameWithOverwrite(Directory dir, String fileName, String toName) throws IOException {
+    Directory baseDir = getBaseDir(dir);
+    if (baseDir instanceof FSDirectory) {
+      Path path = ((FSDirectory) baseDir).getDirectory().toAbsolutePath();
+      try {
+      Files.move(FileSystems.getDefault().getPath(path.toString(), fileName),
+          FileSystems.getDefault().getPath(path.toString(), toName), StandardCopyOption.ATOMIC_MOVE, StandardCopyOption.REPLACE_EXISTING);
+      } catch (AtomicMoveNotSupportedException e) {
+        Files.move(FileSystems.getDefault().getPath(path.toString(), fileName),
+            FileSystems.getDefault().getPath(path.toString(), toName), StandardCopyOption.REPLACE_EXISTING);
+      }
+    } else {
+      super.renameWithOverwrite(dir, fileName, toName);
+    }
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/96ed221f/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
index bdbd4e7..8bdd2b8 100644
--- a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
+++ b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
@@ -685,15 +685,19 @@ public class IndexFetcher {
         sb = readToStringBuilder(replicationTime, props.getProperty(REPLICATION_FAILED_AT_LIST));
         props.setProperty(REPLICATION_FAILED_AT_LIST, sb.toString());
       }
-
-      final IndexOutput out = dir.createOutput(REPLICATION_PROPERTIES, DirectoryFactory.IOCONTEXT_NO_CACHE);
+      
+      
+      String tmpFileName = REPLICATION_PROPERTIES + "." + System.nanoTime();
+      final IndexOutput out = dir.createOutput(tmpFileName, DirectoryFactory.IOCONTEXT_NO_CACHE);
       Writer outFile = new OutputStreamWriter(new PropertiesOutputStream(out), StandardCharsets.UTF_8);
       try {
         props.store(outFile, "Replication details");
-        dir.sync(Collections.singleton(REPLICATION_PROPERTIES));
+        dir.sync(Collections.singleton(tmpFileName));
       } finally {
         IOUtils.closeQuietly(outFile);
       }
+      
+      solrCore.getDirectoryFactory().renameWithOverwrite(dir, tmpFileName, REPLICATION_PROPERTIES);
     } catch (Exception e) {
       LOG.warn("Exception while updating statistics", e);
     } finally {
@@ -1206,24 +1210,23 @@ public class IndexFetcher {
           IOUtils.closeQuietly(is);
         }
       }
-      try {
-        dir.deleteFile(IndexFetcher.INDEX_PROPERTIES);
-      } catch (IOException e) {
-        // no problem
-      }
-      final IndexOutput out = dir.createOutput(IndexFetcher.INDEX_PROPERTIES, DirectoryFactory.IOCONTEXT_NO_CACHE);
+
+      String tmpFileName = IndexFetcher.INDEX_PROPERTIES + "." + System.nanoTime();
+      final IndexOutput out = dir.createOutput(tmpFileName, DirectoryFactory.IOCONTEXT_NO_CACHE);
       p.put("index", tmpIdxDirName);
       Writer os = null;
       try {
         os = new OutputStreamWriter(new PropertiesOutputStream(out), StandardCharsets.UTF_8);
-        p.store(os, IndexFetcher.INDEX_PROPERTIES);
-        dir.sync(Collections.singleton(INDEX_PROPERTIES));
+        p.store(os, tmpFileName);
+        dir.sync(Collections.singleton(tmpFileName));
       } catch (Exception e) {
         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
             "Unable to write " + IndexFetcher.INDEX_PROPERTIES, e);
       } finally {
         IOUtils.closeQuietly(os);
       }
+      
+      solrCore.getDirectoryFactory().renameWithOverwrite(dir, tmpFileName, IndexFetcher.INDEX_PROPERTIES);
       return true;
 
     } catch (IOException e1) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/96ed221f/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java b/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
index 08c462b..685ef99 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
@@ -35,6 +35,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Date;
+import java.util.List;
 import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
@@ -304,7 +305,11 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
     // check details on the slave a couple of times before & after fetching
     for (int i = 0; i < 3; i++) {
       NamedList<Object> details = getDetails(slaveClient);
-      
+      List replicatedAtCount = (List) ((NamedList) details.get("slave")).get("indexReplicatedAtList");
+      if (i > 0) {
+        assertEquals(i, replicatedAtCount.size());
+      }
+
       assertEquals("slave isMaster?", 
                    "false", details.get("isMaster"));
       assertEquals("slave isSlave?",