You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sa...@apache.org on 2016/08/02 01:07:21 UTC

[01/40] lucene-solr:apiv2: LUCENE-7381: add point based DoubleRangeField and RangeFieldQuery for indexing and querying on Ranges up to 4 dimensions

Repository: lucene-solr
Updated Branches:
  refs/heads/apiv2 003f9b74a -> 1d131592c


LUCENE-7381: add point based DoubleRangeField and RangeFieldQuery for indexing and querying on Ranges up to 4 dimensions


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

Branch: refs/heads/apiv2
Commit: f5964ec6e45adc8fc7644bbe60d9e5966bf11ab7
Parents: d4cb52f
Author: Nicholas Knize <nk...@gmail.com>
Authored: Fri Jul 15 14:54:44 2016 -0500
Committer: Nicholas Knize <nk...@gmail.com>
Committed: Wed Jul 20 21:29:06 2016 -0500

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   3 +
 .../lucene/document/DoubleRangeField.java       | 262 ++++++++++++
 .../apache/lucene/document/RangeFieldQuery.java | 313 ++++++++++++++
 .../lucene/document/TestDoubleRangeField.java   |  67 +++
 .../search/BaseRangeFieldQueryTestCase.java     | 403 +++++++++++++++++++
 .../search/TestDoubleRangeFieldQueries.java     | 106 +++++
 6 files changed, 1154 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f5964ec6/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 432e1d2..6d66355 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -38,6 +38,9 @@ API Changes
 
 New Features
 
+* LUCENE-7381: Add point based DoubleRangeField and RangeFieldQuery for
+  indexing and querying on Ranges up to 4 dimensions (Nick Knize)
+
 * LUCENE-7302: IndexWriter methods that change the index now return a
   long "sequence number" indicating the effective equivalent
   single-threaded execution order (Mike McCandless)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f5964ec6/lucene/sandbox/src/java/org/apache/lucene/document/DoubleRangeField.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/DoubleRangeField.java b/lucene/sandbox/src/java/org/apache/lucene/document/DoubleRangeField.java
new file mode 100644
index 0000000..2af8697
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/DoubleRangeField.java
@@ -0,0 +1,262 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.document;
+
+import org.apache.lucene.document.RangeFieldQuery.QueryType;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.NumericUtils;
+
+/**
+ * An indexed Double Range field.
+ * <p>
+ * This field indexes dimensional ranges defined as min/max pairs. It supports
+ * up to a maximum of 4 dimensions (indexed as 8 numeric values). With 1 dimension representing a single double range,
+ * 2 dimensions representing a bounding box, 3 dimensions a bounding cube, and 4 dimensions a tesseract.
+ * <p>
+ * Multiple values for the same field in one document is supported, and open ended ranges can be defined using
+ * {@code Double.NEGATIVE_INFINITY} and {@code Double.POSITIVE_INFINITY}.
+ *
+ * <p>
+ * This field defines the following static factory methods for common search operations over double ranges:
+ * <ul>
+ *   <li>{@link #newIntersectsQuery newIntersectsQuery()} matches ranges that intersect the defined search range.
+ *   <li>{@link #newWithinQuery newWithinQuery()} matches ranges that are within the defined search range.
+ *   <li>{@link #newContainsQuery newContainsQuery()} matches ranges that contain the defined search range.
+ * </ul>
+ */
+public class DoubleRangeField extends Field {
+  /** stores double values so number of bytes is 8 */
+  public static final int BYTES = Double.BYTES;
+
+  /**
+   * Create a new DoubleRangeField type, from min/max parallel arrays
+   *
+   * @param name field name. must not be null.
+   * @param min range min values; each entry is the min value for the dimension
+   * @param max range max values; each entry is the max value for the dimension
+   */
+  public DoubleRangeField(String name, final double[] min, final double[] max) {
+    super(name, getType(min.length));
+    setRangeValues(min, max);
+  }
+
+  /** set the field type */
+  private static FieldType getType(int dimensions) {
+    if (dimensions > 4) {
+      throw new IllegalArgumentException("DoubleRangeField does not support greater than 4 dimensions");
+    }
+
+    FieldType ft = new FieldType();
+    // dimensions is set as 2*dimension size (min/max per dimension)
+    ft.setDimensions(dimensions*2, BYTES);
+    ft.freeze();
+    return ft;
+  }
+
+  /**
+   * Changes the values of the field.
+   * @param min array of min values. (accepts {@code Double.NEGATIVE_INFINITY})
+   * @param max array of max values. (accepts {@code Double.POSITIVE_INFINITY})
+   * @throws IllegalArgumentException if {@code min} or {@code max} is invalid
+   */
+  public void setRangeValues(double[] min, double[] max) {
+    checkArgs(min, max);
+    if (min.length*2 != type.pointDimensionCount() || max.length*2 != type.pointDimensionCount()) {
+      throw new IllegalArgumentException("field (name=" + name + ") uses " + type.pointDimensionCount()/2
+          + " dimensions; cannot change to (incoming) " + min.length + " dimensions");
+    }
+
+    final byte[] bytes;
+    if (fieldsData == null) {
+      bytes = new byte[BYTES*2*min.length];
+      fieldsData = new BytesRef(bytes);
+    } else {
+      bytes = ((BytesRef)fieldsData).bytes;
+    }
+    verifyAndEncode(min, max, bytes);
+  }
+
+  /** validate the arguments */
+  private static void checkArgs(final double[] min, final double[] max) {
+    if (min == null || max == null || min.length == 0 || max.length == 0) {
+      throw new IllegalArgumentException("min/max range values cannot be null or empty");
+    }
+    if (min.length != max.length) {
+      throw new IllegalArgumentException("min/max ranges must agree");
+    }
+    if (min.length > 4) {
+      throw new IllegalArgumentException("DoubleRangeField does not support greater than 4 dimensions");
+    }
+  }
+
+  /**
+   * Encodes the min, max ranges into a byte array
+   */
+  private static byte[] encode(double[] min, double[] max) {
+    checkArgs(min, max);
+    byte[] b = new byte[BYTES*2*min.length];
+    verifyAndEncode(min, max, b);
+    return b;
+  }
+
+  /**
+   * encode the ranges into a sortable byte array ({@code Double.NaN} not allowed)
+   * <p>
+   * example for 4 dimensions (8 bytes per dimension value):
+   * minD1 ... minD4 | maxD1 ... maxD4
+   */
+  static void verifyAndEncode(double[] min, double[] max, byte[] bytes) {
+    for (int d=0,i=0,j=min.length*BYTES; d<min.length; ++d, i+=BYTES, j+=BYTES) {
+      if (Double.isNaN(min[d])) {
+        throw new IllegalArgumentException("invalid min value (" + Double.NaN + ")" + " in DoubleRangeField");
+      }
+      if (Double.isNaN(max[d])) {
+        throw new IllegalArgumentException("invalid max value (" + Double.NaN + ")" + " in DoubleRangeField");
+      }
+      if (min[d] > max[d]) {
+        throw new IllegalArgumentException("min value (" + min[d] + ") is greater than max value (" + max[d] + ")");
+      }
+      encode(min[d], bytes, i);
+      encode(max[d], bytes, j);
+    }
+  }
+
+  /** encode the given value into the byte array at the defined offset */
+  private static void encode(double val, byte[] bytes, int offset) {
+    NumericUtils.longToSortableBytes(NumericUtils.doubleToSortableLong(val), bytes, offset);
+  }
+
+  /**
+   * Get the min value for the given dimension
+   * @param dimension the dimension, always positive
+   * @return the decoded min value
+   */
+  public double getMin(int dimension) {
+    if (dimension < 0 || dimension >= type.pointDimensionCount()/2) {
+      throw new IllegalArgumentException("dimension request (" + dimension +
+          ") out of bounds for field (name=" + name + " dimensions=" + type.pointDimensionCount()/2 + "). ");
+    }
+    return decodeMin(((BytesRef)fieldsData).bytes, dimension);
+  }
+
+  /**
+   * Get the max value for the given dimension
+   * @param dimension the dimension, always positive
+   * @return the decoded max value
+   */
+  public double getMax(int dimension) {
+    if (dimension < 0 || dimension >= type.pointDimensionCount()/2) {
+      throw new IllegalArgumentException("dimension request (" + dimension +
+          ") out of bounds for field (name=" + name + " dimensions=" + type.pointDimensionCount()/2 + "). ");
+    }
+    return decodeMax(((BytesRef)fieldsData).bytes, dimension);
+  }
+
+  /** decodes the min value (for the defined dimension) from the encoded input byte array */
+  static double decodeMin(byte[] b, int dimension) {
+    int offset = dimension*BYTES;
+    return NumericUtils.sortableLongToDouble(NumericUtils.sortableBytesToLong(b, offset));
+  }
+
+  /** decodes the max value (for the defined dimension) from the encoded input byte array */
+  static double decodeMax(byte[] b, int dimension) {
+    int offset = b.length/2 + dimension*BYTES;
+    return NumericUtils.sortableLongToDouble(NumericUtils.sortableBytesToLong(b, offset));
+  }
+
+  /**
+   * Create a query for matching indexed ranges that intersect the defined range.
+   * @param field field name. must not be null.
+   * @param min array of min values. (accepts {@code Double.NEGATIVE_INFINITY})
+   * @param max array of max values. (accepts {@code Double.POSITIVE_INFINITY})
+   * @return query for matching intersecting ranges (overlap, within, or contains)
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newIntersectsQuery(String field, final double[] min, final double[] max) {
+    return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.INTERSECTS) {
+      @Override
+      protected String toString(byte[] ranges, int dimension) {
+        return DoubleRangeField.toString(ranges, dimension);
+      }
+    };
+  }
+
+  /**
+   * Create a query for matching indexed ranges that contain the defined range.
+   * @param field field name. must not be null.
+   * @param min array of min values. (accepts {@code Double.MIN_VALUE})
+   * @param max array of max values. (accepts {@code Double.MAX_VALUE})
+   * @return query for matching ranges that contain the defined range
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newContainsQuery(String field, final double[] min, final double[] max) {
+    return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.CONTAINS) {
+      @Override
+      protected String toString(byte[] ranges, int dimension) {
+        return DoubleRangeField.toString(ranges, dimension);
+      }
+    };
+  }
+
+  /**
+   * Create a query for matching indexed ranges that are within the defined range.
+   * @param field field name. must not be null.
+   * @param min array of min values. (accepts {@code Double.MIN_VALUE})
+   * @param max array of max values. (accepts {@code Double.MAX_VALUE})
+   * @return query for matching ranges within the defined range
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newWithinQuery(String field, final double[] min, final double[] max) {
+    checkArgs(min, max);
+    return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.WITHIN) {
+      @Override
+      protected String toString(byte[] ranges, int dimension) {
+        return DoubleRangeField.toString(ranges, dimension);
+      }
+    };
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append(getClass().getSimpleName());
+    sb.append(" <");
+    sb.append(name);
+    sb.append(':');
+    byte[] b = ((BytesRef)fieldsData).bytes;
+    toString(b, 0);
+    for (int d=1; d<type.pointDimensionCount(); ++d) {
+      sb.append(' ');
+      toString(b, d);
+    }
+    sb.append('>');
+
+    return sb.toString();
+  }
+
+  /**
+   * Returns the String representation for the range at the given dimension
+   * @param ranges the encoded ranges, never null
+   * @param dimension the dimension of interest
+   * @return The string representation for the range at the provided dimension
+   */
+  private static String toString(byte[] ranges, int dimension) {
+    return "[" + Double.toString(decodeMin(ranges, dimension)) + " : "
+        + Double.toString(decodeMax(ranges, dimension)) + "]";
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f5964ec6/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java b/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java
new file mode 100644
index 0000000..36de9b2
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java
@@ -0,0 +1,313 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.document;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Objects;
+
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.index.PointValues.Relation;
+import org.apache.lucene.index.PointValues.IntersectVisitor;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSet;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.DocIdSetBuilder;
+import org.apache.lucene.util.StringHelper;
+
+/**
+ * Query class for searching {@code RangeField} types by a defined {@link Relation}.
+ */
+abstract class RangeFieldQuery extends Query {
+  /** field name */
+  final String field;
+  /** query relation
+   * intersects: {@code CELL_CROSSES_QUERY},
+   * contains: {@code CELL_CONTAINS_QUERY},
+   * within: {@code CELL_WITHIN_QUERY} */
+  final QueryType queryType;
+  /** number of dimensions - max 4 */
+  final int numDims;
+  /** ranges encoded as a sortable byte array */
+  final byte[] ranges;
+  /** number of bytes per dimension */
+  final int bytesPerDim;
+
+  /** Used by {@code RangeFieldQuery} to check how each internal or leaf node relates to the query. */
+  enum QueryType {
+    /** Use this for intersects queries. */
+    INTERSECTS,
+    /** Use this for within queries. */
+    WITHIN,
+    /** Use this for contains */
+    CONTAINS
+  }
+
+  /**
+   * Create a query for searching indexed ranges that match the provided relation.
+   * @param field field name. must not be null.
+   * @param ranges encoded range values; this is done by the {@code RangeField} implementation
+   * @param queryType the query relation
+   */
+  RangeFieldQuery(String field, final byte[] ranges, final int numDims, final QueryType queryType) {
+    checkArgs(field, ranges, numDims);
+    if (queryType == null) {
+      throw new IllegalArgumentException("Query type cannot be null");
+    }
+    this.field = field;
+    this.queryType = queryType;
+    this.numDims = numDims;
+    this.ranges = ranges;
+    this.bytesPerDim = ranges.length / (2*numDims);
+  }
+
+  /** check input arguments */
+  private static void checkArgs(String field, final byte[] ranges, final int numDims) {
+    if (field == null) {
+      throw new IllegalArgumentException("field must not be null");
+    }
+    if (numDims > 4) {
+      throw new IllegalArgumentException("dimension size cannot be greater than 4");
+    }
+    if (ranges == null || ranges.length == 0) {
+      throw new IllegalArgumentException("encoded ranges cannot be null or empty");
+    }
+  }
+
+  /** Check indexed field info against the provided query data. */
+  private void checkFieldInfo(FieldInfo fieldInfo) {
+    if (fieldInfo.getPointDimensionCount()/2 != numDims) {
+      throw new IllegalArgumentException("field=\"" + field + "\" was indexed with numDims="
+          + fieldInfo.getPointDimensionCount()/2 + " but this query has numDims=" + numDims);
+    }
+  }
+
+  @Override
+  public final Weight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException {
+    return new ConstantScoreWeight(this, boost) {
+      final RangeFieldComparator comparator = new RangeFieldComparator();
+      private DocIdSet buildMatchingDocIdSet(LeafReader reader, PointValues values) throws IOException {
+        DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc(), values, field);
+        values.intersect(field,
+            new IntersectVisitor() {
+              DocIdSetBuilder.BulkAdder adder;
+              @Override
+              public void grow(int count) {
+                adder = result.grow(count);
+              }
+              @Override
+              public void visit(int docID) throws IOException {
+                adder.add(docID);
+              }
+              @Override
+              public void visit(int docID, byte[] leaf) throws IOException {
+                // add the document iff:
+                if (// target is within cell and queryType is INTERSECTS or CONTAINS:
+                    (comparator.isWithin(leaf) && queryType != QueryType.WITHIN)
+                    // target contains cell and queryType is INTERSECTS or WITHIN:
+                    || (comparator.contains(leaf) && queryType != QueryType.CONTAINS)
+                    // target is not disjoint (crosses) and queryType is INTERSECTS
+                    || (comparator.isDisjoint(leaf) == false && queryType == QueryType.INTERSECTS)) {
+                  adder.add(docID);
+                }
+              }
+              @Override
+              public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+                byte[] node = getInternalRange(minPackedValue, maxPackedValue);
+                // compute range relation for BKD traversal
+                if (comparator.isDisjoint(node)) {
+                  return Relation.CELL_OUTSIDE_QUERY;
+                } else if (comparator.contains(node)) {
+                  // target contains cell; add iff queryType is not a CONTAINS query:
+                  return (queryType == QueryType.CONTAINS) ? Relation.CELL_OUTSIDE_QUERY : Relation.CELL_INSIDE_QUERY;
+                } else if (comparator.isWithin(node)) {
+                  // target within cell; continue traversing:
+                  return Relation.CELL_CROSSES_QUERY;
+                }
+                // target intersects cell; continue traversing:
+                return Relation.CELL_CROSSES_QUERY;
+              }
+            });
+        return result.build();
+      }
+
+      @Override
+      public Scorer scorer(LeafReaderContext context) throws IOException {
+        LeafReader reader = context.reader();
+        PointValues values = reader.getPointValues();
+        if (values == null) {
+          // no docs in this segment indexed any ranges
+          return null;
+        }
+        FieldInfo fieldInfo = reader.getFieldInfos().fieldInfo(field);
+        if (fieldInfo == null) {
+          // no docs in this segment indexed this field
+        }
+        checkFieldInfo(fieldInfo);
+        boolean allDocsMatch = true;
+        if (values.getDocCount(field) == reader.maxDoc()) {
+          // if query crosses, docs need to be further scrutinized
+          byte[] range = getInternalRange(values.getMinPackedValue(field), values.getMaxPackedValue(field));
+          // if the internal node is not contained by the query, all docs do not match
+          if (((comparator.contains(range) && queryType == QueryType.CONTAINS)) == false) {
+            allDocsMatch = false;
+          }
+        } else {
+          allDocsMatch = false;
+        }
+
+        DocIdSetIterator iterator = allDocsMatch == true ?
+            DocIdSetIterator.all(reader.maxDoc()) : buildMatchingDocIdSet(reader, values).iterator();
+        return new ConstantScoreScorer(this, score(), iterator);
+      }
+
+      /** get an encoded byte representation of the internal node; this is
+       *  the lower half of the min array and the upper half of the max array */
+      private byte[] getInternalRange(byte[] min, byte[] max) {
+        byte[] range = new byte[min.length];
+        final int dimSize = numDims * bytesPerDim;
+        System.arraycopy(min, 0, range, 0, dimSize);
+        System.arraycopy(max, dimSize, range, dimSize, dimSize);
+        return range;
+      }
+    };
+  }
+
+  /**
+   * RangeFieldComparator class provides the core comparison logic for accepting or rejecting indexed
+   * {@code RangeField} types based on the defined query range and relation.
+   */
+  class RangeFieldComparator {
+    /** check if the query is outside the candidate range */
+    private boolean isDisjoint(final byte[] range) {
+      for (int d=0; d<numDims; ++d) {
+        if (compareMinMax(range, d) > 0 || compareMaxMin(range, d) < 0) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    /** check if query is within candidate range */
+    private boolean isWithin(final byte[] range) {
+      for (int d=0; d<numDims; ++d) {
+        if (compareMinMin(range, d) < 0 || compareMaxMax(range, d) > 0) {
+          return false;
+        }
+      }
+      return true;
+    }
+
+    /** check if query contains candidate range */
+    private boolean contains(final byte[] range) {
+      for (int d=0; d<numDims; ++d) {
+        if (compareMinMin(range, d) > 0 || compareMaxMax(range, d) < 0) {
+          return false;
+        }
+      }
+      return true;
+    }
+
+    /** compare the encoded min value (for the defined query dimension) with the encoded min value in the byte array */
+    private int compareMinMin(byte[] b, int dimension) {
+      // convert dimension to offset:
+      dimension *= bytesPerDim;
+      return StringHelper.compare(bytesPerDim, ranges, dimension, b, dimension);
+    }
+
+    /** compare the encoded min value (for the defined query dimension) with the encoded max value in the byte array */
+    private int compareMinMax(byte[] b, int dimension) {
+      // convert dimension to offset:
+      dimension *= bytesPerDim;
+      return StringHelper.compare(bytesPerDim, ranges, dimension, b, numDims * bytesPerDim + dimension);
+    }
+
+    /** compare the encoded max value (for the defined query dimension) with the encoded min value in the byte array */
+    private int compareMaxMin(byte[] b, int dimension) {
+      // convert dimension to offset:
+      dimension *= bytesPerDim;
+      return StringHelper.compare(bytesPerDim, ranges, numDims * bytesPerDim + dimension, b, dimension);
+    }
+
+    /** compare the encoded max value (for the defined query dimension) with the encoded max value in the byte array */
+    private int compareMaxMax(byte[] b, int dimension) {
+      // convert dimension to max offset:
+      dimension = numDims * bytesPerDim + dimension * bytesPerDim;
+      return StringHelper.compare(bytesPerDim, ranges, dimension, b, dimension);
+    }
+  }
+
+  @Override
+  public int hashCode() {
+    int hash = classHash();
+    hash = 31 * hash + field.hashCode();
+    hash = 31 * hash + numDims;
+    hash = 31 * hash + queryType.hashCode();
+    hash = 31 * hash + Arrays.hashCode(ranges);
+
+    return hash;
+  }
+
+  @Override
+  public final boolean equals(Object o) {
+    return sameClassAs(o) &&
+        equalsTo(getClass().cast(o));
+  }
+
+  protected boolean equalsTo(RangeFieldQuery other) {
+    return Objects.equals(field, other.field) &&
+        numDims == other.numDims &&
+        Arrays.equals(ranges, other.ranges) &&
+        other.queryType == queryType;
+  }
+
+  @Override
+  public String toString(String field) {
+    StringBuilder sb = new StringBuilder();
+    if (this.field.equals(field) == false) {
+      sb.append(this.field);
+      sb.append(':');
+    }
+    sb.append("<ranges:");
+    sb.append(toString(ranges, 0));
+    for (int d=1; d<numDims; ++d) {
+      sb.append(' ');
+      sb.append(toString(ranges, d));
+    }
+    sb.append('>');
+
+    return sb.toString();
+  }
+
+  /**
+   * Returns a string of a single value in a human-readable format for debugging.
+   * This is used by {@link #toString()}.
+   *
+   * @param dimension dimension of the particular value
+   * @param ranges encoded ranges, never null
+   * @return human readable value for debugging
+   */
+  protected abstract String toString(byte[] ranges, int dimension);
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f5964ec6/lucene/sandbox/src/test/org/apache/lucene/document/TestDoubleRangeField.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestDoubleRangeField.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestDoubleRangeField.java
new file mode 100644
index 0000000..188aab6
--- /dev/null
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestDoubleRangeField.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.document;
+
+import org.apache.lucene.util.LuceneTestCase;
+
+/**
+ * Random testing for RangeField type.
+ **/
+public class TestDoubleRangeField extends LuceneTestCase {
+  private static final String FIELD_NAME = "rangeField";
+
+  /** test illegal NaN range values */
+  public void testIllegalNaNValues() {
+    Document doc = new Document();
+    IllegalArgumentException expected;
+
+    expected = expectThrows(IllegalArgumentException.class, () ->
+        doc.add(new DoubleRangeField(FIELD_NAME, new double[] {Double.NaN}, new double[] {5})));
+    assertTrue(expected.getMessage().contains("invalid min value"));
+
+    expected = expectThrows(IllegalArgumentException.class, () ->
+        doc.add(new DoubleRangeField(FIELD_NAME, new double[] {5}, new double[] {Double.NaN})));
+    assertTrue(expected.getMessage().contains("invalid max value"));
+  }
+
+  /** min/max array sizes must agree */
+  public void testUnevenArrays() {
+    Document doc = new Document();
+    IllegalArgumentException expected;
+    expected = expectThrows(IllegalArgumentException.class, () ->
+        doc.add(new DoubleRangeField(FIELD_NAME, new double[] {5, 6}, new double[] {5})));
+    assertTrue(expected.getMessage().contains("min/max ranges must agree"));
+  }
+
+  /** dimensions greater than 4 not supported */
+  public void testOversizeDimensions() {
+    Document doc = new Document();
+    IllegalArgumentException expected;
+    expected = expectThrows(IllegalArgumentException.class, () ->
+        doc.add(new DoubleRangeField(FIELD_NAME, new double[] {1, 2, 3, 4, 5}, new double[] {5})));
+    assertTrue(expected.getMessage().contains("does not support greater than 4 dimensions"));
+  }
+
+  /** min cannot be greater than max */
+  public void testMinGreaterThanMax() {
+    Document doc = new Document();
+    IllegalArgumentException expected;
+    expected = expectThrows(IllegalArgumentException.class, () ->
+      doc.add(new DoubleRangeField(FIELD_NAME, new double[] {3, 4}, new double[] {1, 2})));
+    assertTrue(expected.getMessage().contains("is greater than max value"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f5964ec6/lucene/sandbox/src/test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java b/lucene/sandbox/src/test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java
new file mode 100644
index 0000000..dadb588
--- /dev/null
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java
@@ -0,0 +1,403 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.MultiDocValues;
+import org.apache.lucene.index.MultiFields;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.SerialMergeScheduler;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LuceneTestCase;
+
+/**
+ * Abstract class to do basic tests for a RangeField query.
+ */
+public abstract class BaseRangeFieldQueryTestCase extends LuceneTestCase {
+  protected abstract Field newRangeField(double[] min, double[] max);
+
+  protected abstract Query newIntersectsQuery(double[] min, double[] max);
+
+  protected abstract Query newContainsQuery(double[] min, double[] max);
+
+  protected abstract Query newWithinQuery(double[] min, double[] max);
+
+  protected int dimension() {
+    return random().nextInt(4) + 1;
+  }
+
+  public void testRandomTiny() throws Exception {
+    // Make sure single-leaf-node case is OK:
+    doTestRandom(10, false);
+  }
+
+  public void testRandomMedium() throws Exception {
+    doTestRandom(10000, false);
+  }
+
+  @Nightly
+  public void testRandomBig() throws Exception {
+    doTestRandom(200000, false);
+  }
+
+  public void testMultiValued() throws Exception {
+    doTestRandom(10000, true);
+  }
+
+  private void doTestRandom(int count, boolean multiValued) throws Exception {
+    int numDocs = atLeast(count);
+    int dimensions = dimension();
+
+    if (VERBOSE) {
+      System.out.println("TEST: numDocs=" + numDocs);
+    }
+
+    Box[][] boxes = new Box[numDocs][];
+
+    boolean haveRealDoc = true;
+
+    nextdoc: for (int id=0; id<numDocs; ++id) {
+      int x = random().nextInt(20);
+      if (boxes[id] == null) {
+        boxes[id] = new Box[] {nextBox(dimensions)};
+      }
+      if (x == 17) {
+        // dome docs don't have a box:
+        boxes[id][0].min[0] = Double.NaN;
+        if (VERBOSE) {
+          System.out.println("  id=" + id + " is missing");
+        }
+        continue;
+      }
+
+      if (multiValued == true && random().nextBoolean()) {
+        // randomly add multi valued documents (up to 2 fields)
+        int n = random().nextInt(2) + 1;
+        boxes[id] = new Box[n];
+        for (int i=0; i<n; ++i) {
+          boxes[id][i] = nextBox(dimensions);
+        }
+      }
+
+      if (id > 0 && x < 9 && haveRealDoc) {
+        int oldID;
+        int i=0;
+        // don't step on missing boxes:
+        while (true) {
+          oldID = random().nextInt(id);
+          if (Double.isNaN(boxes[oldID][0].min[0]) == false) {
+            break;
+          } else if (++i > id) {
+            continue nextdoc;
+          }
+        }
+
+        if (x == dimensions*2) {
+          // Fully identical box (use first box in case current is multivalued but old is not)
+          for (int d=0; d<dimensions; ++d) {
+            boxes[id][0].min[d] = boxes[oldID][0].min[d];
+            boxes[id][0].max[d] = boxes[oldID][0].max[d];
+          }
+          if (VERBOSE) {
+            System.out.println("  id=" + id + " box=" + boxes[id] + " (same box as doc=" + oldID + ")");
+          }
+        } else {
+          for (int m = 0, even = dimensions % 2; m < dimensions * 2; ++m) {
+            if (x == m) {
+              int d = (int)Math.floor(m/2);
+              // current could be multivalue but old may not be, so use first box
+              if (even == 0) {
+                boxes[id][0].setVal(d, boxes[oldID][0].min[d]);
+                if (VERBOSE) {
+                  System.out.println("  id=" + id + " box=" + boxes[id] + " (same min[" + d + "] as doc=" + oldID + ")");
+                }
+              } else {
+                boxes[id][0].setVal(d, boxes[oldID][0].max[d]);
+                if (VERBOSE) {
+                  System.out.println("  id=" + id + " box=" + boxes[id] + " (same max[" + d + "] as doc=" + oldID + ")");
+                }
+              }
+            }
+          }
+        }
+      }
+    }
+    verify(boxes);
+  }
+
+  private void verify(Box[][] boxes) throws Exception {
+    IndexWriterConfig iwc = newIndexWriterConfig();
+    // Else seeds may not reproduce:
+    iwc.setMergeScheduler(new SerialMergeScheduler());
+    // Else we can get O(N^2) merging
+    int mbd = iwc.getMaxBufferedDocs();
+    if (mbd != -1 && mbd < boxes.length/100) {
+      iwc.setMaxBufferedDocs(boxes.length/100);
+    }
+    Directory dir;
+    if (boxes.length > 50000) {
+      dir = newFSDirectory(createTempDir(getClass().getSimpleName()));
+    } else {
+      dir = newDirectory();
+    }
+
+    Set<Integer> deleted = new HashSet<>();
+    IndexWriter w = new IndexWriter(dir, iwc);
+    for (int id=0; id < boxes.length; ++id) {
+      Document doc = new Document();
+      doc.add(newStringField("id", ""+id, Field.Store.NO));
+      doc.add(new NumericDocValuesField("id", id));
+      if (Double.isNaN(boxes[id][0].min[0]) == false) {
+        for (int n=0; n<boxes[id].length; ++n) {
+          doc.add(newRangeField(boxes[id][n].min, boxes[id][n].max));
+        }
+      }
+      w.addDocument(doc);
+      if (id > 0 && random().nextInt(100) == 1) {
+        int idToDelete = random().nextInt(id);
+        w.deleteDocuments(new Term("id", ""+idToDelete));
+        deleted.add(idToDelete);
+        if (VERBOSE) {
+          System.out.println("  delete id=" + idToDelete);
+        }
+      }
+    }
+
+    if (random().nextBoolean()) {
+      w.forceMerge(1);
+    }
+    final IndexReader r = DirectoryReader.open(w);
+    w.close();
+    IndexSearcher s = newSearcher(r);
+
+    int dimensions = boxes[0][0].min.length;
+    int iters = atLeast(25);
+    NumericDocValues docIDToID = MultiDocValues.getNumericValues(r, "id");
+    Bits liveDocs = MultiFields.getLiveDocs(s.getIndexReader());
+    int maxDoc = s.getIndexReader().maxDoc();
+
+    for (int iter=0; iter<iters; ++iter) {
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter + " s=" + s);
+      }
+
+      // occasionally test open ended bounding boxes
+      Box queryBox = nextBox(dimensions);
+      int rv = random().nextInt(3);
+      Query query;
+      Box.QueryType queryType;
+      if (rv == 0) {
+        queryType = Box.QueryType.INTERSECTS;
+        query = newIntersectsQuery(queryBox.min, queryBox.max);
+      } else if (rv == 1)  {
+        queryType = Box.QueryType.CONTAINS;
+        query = newContainsQuery(queryBox.min, queryBox.max);
+      } else {
+        queryType = Box.QueryType.WITHIN;
+        query = newWithinQuery(queryBox.min, queryBox.max);
+      }
+
+      if (VERBOSE) {
+        System.out.println("  query=" + query);
+      }
+
+      final FixedBitSet hits = new FixedBitSet(maxDoc);
+      s.search(query, new SimpleCollector() {
+        private int docBase;
+
+        @Override
+        public void collect(int doc) {
+          hits.set(docBase + doc);
+        }
+
+        @Override
+        protected void doSetNextReader(LeafReaderContext context) throws IOException {
+          docBase = context.docBase;
+        }
+
+        @Override
+        public boolean needsScores() { return false; }
+      });
+
+      for (int docID=0; docID<maxDoc; ++docID) {
+        int id = (int) docIDToID.get(docID);
+        boolean expected;
+        if (liveDocs != null && liveDocs.get(docID) == false) {
+          // document is deleted
+          expected = false;
+        } else if (Double.isNaN(boxes[id][0].min[0])) {
+          expected = false;
+        } else {
+          expected = expectedResult(queryBox, boxes[id], queryType);
+        }
+
+        if (hits.get(docID) != expected) {
+          StringBuilder b = new StringBuilder();
+          if (expected == true) {
+            b.append("FAILS: id=" + id + (boxes[id].length > 1 ? " (MultiValue) " : " ") + "should match but did not\n");
+          } else {
+            b.append("FAIL: id=" + id + " should not match but did\n");
+          }
+          b.append(" queryBox=" + queryBox + "\n");
+          b.append(" box" + ((boxes[id].length > 1) ? "es=" : "=" ) + boxes[id][0]);
+          for (int n=1; n<boxes[id].length; ++n) {
+            b.append(", ");
+            b.append(boxes[id][n]);
+          }
+          b.append("\n queryType=" + queryType + "\n");
+          b.append(" deleted?=" + (liveDocs != null && liveDocs.get(docID) == false));
+          fail("wrong hit (first of possibly more):\n\n" + b);
+        }
+      }
+    }
+    IOUtils.close(r, dir);
+  }
+
+  protected boolean expectedResult(Box queryBox, Box[] box, Box.QueryType queryType) {
+    for (int i=0; i<box.length; ++i) {
+      if (expectedBBoxQueryResult(queryBox, box[i], queryType) == true) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  protected boolean expectedBBoxQueryResult(Box queryBox, Box box, Box.QueryType queryType) {
+    Box.QueryType relation = box.relate(queryBox);
+    if (queryType == Box.QueryType.INTERSECTS) {
+      return relation != null;
+    }
+    return relation == queryType;
+  }
+
+  protected double nextDoubleInternal() {
+    if (rarely()) {
+      return random().nextBoolean() ? Double.POSITIVE_INFINITY : Double.NEGATIVE_INFINITY;
+    }
+    double max = 100 / 2;
+    return (max + max) * random().nextDouble() - max;
+  }
+
+  protected Box nextBox(int dimensions) {
+    double[] min = new double[dimensions];
+    double[] max = new double[dimensions];
+
+    for (int d=0; d<dimensions; ++d) {
+      min[d] = nextDoubleInternal();
+      max[d] = nextDoubleInternal();
+    }
+
+    return new Box(min, max);
+  }
+
+  protected static class Box {
+    double[] min;
+    double[] max;
+
+    enum QueryType { INTERSECTS, WITHIN, CONTAINS }
+
+    Box(double[] min, double[] max) {
+      assert min != null && max != null && min.length > 0 && max.length > 0
+          : "test box: min/max cannot be null or empty";
+      assert min.length == max.length : "test box: min/max length do not agree";
+      this.min = new double[min.length];
+      this.max = new double[max.length];
+      for (int d=0; d<min.length; ++d) {
+        this.min[d] = Math.min(min[d], max[d]);
+        this.max[d] = Math.max(min[d], max[d]);
+      }
+    }
+
+    protected void setVal(int dimension, double val) {
+      if (val <= min[dimension]) {
+        min[dimension] = val;
+      } else {
+        max[dimension] = val;
+      }
+    }
+
+    QueryType relate(Box other) {
+      // check disjoint
+      for (int d=0; d<this.min.length; ++d) {
+        if (this.min[d] > other.max[d] || this.max[d] < other.min[d]) {
+          // disjoint:
+          return null;
+        }
+      }
+
+      // check within
+      boolean within = true;
+      for (int d=0; d<this.min.length; ++d) {
+        if ((this.min[d] >= other.min[d] && this.max[d] <= other.max[d]) == false) {
+          // not within:
+          within = false;
+          break;
+        }
+      }
+      if (within == true) {
+        return QueryType.WITHIN;
+      }
+
+      // check contains
+      boolean contains = true;
+      for (int d=0; d<this.min.length; ++d) {
+        if ((this.min[d] <= other.min[d] && this.max[d] >= other.max[d]) == false) {
+          // not contains:
+          contains = false;
+          break;
+        }
+      }
+      if (contains == true) {
+        return QueryType.CONTAINS;
+      }
+      return QueryType.INTERSECTS;
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder b = new StringBuilder();
+      b.append("Box(");
+      b.append(min[0]);
+      b.append(" TO ");
+      b.append(max[0]);
+      for (int d=1; d<min.length; ++d) {
+        b.append(", ");
+        b.append(min[d]);
+        b.append(" TO ");
+        b.append(max[d]);
+      }
+      b.append(")");
+
+      return b.toString();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f5964ec6/lucene/sandbox/src/test/org/apache/lucene/search/TestDoubleRangeFieldQueries.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/TestDoubleRangeFieldQueries.java b/lucene/sandbox/src/test/org/apache/lucene/search/TestDoubleRangeFieldQueries.java
new file mode 100644
index 0000000..70857a3
--- /dev/null
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/TestDoubleRangeFieldQueries.java
@@ -0,0 +1,106 @@
+/*
+ * 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.document.Document;
+import org.apache.lucene.document.DoubleRangeField;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.store.Directory;
+
+/**
+ * Random testing for RangeFieldQueries. Testing rigor inspired by {@code BaseGeoPointTestCase}
+ */
+public class TestDoubleRangeFieldQueries extends BaseRangeFieldQueryTestCase {
+  private static final String FIELD_NAME = "rangeField";
+
+  protected DoubleRangeField newRangeField(double[] min, double[] max) {
+    return new DoubleRangeField(FIELD_NAME, min, max);
+  }
+
+  protected Query newIntersectsQuery(double[] min, double[] max) {
+    return DoubleRangeField.newIntersectsQuery(FIELD_NAME, min, max);
+  }
+
+  protected Query newContainsQuery(double[] min, double[] max) {
+    return DoubleRangeField.newContainsQuery(FIELD_NAME, min, max);
+  }
+
+  protected Query newWithinQuery(double[] min, double[] max) {
+    return DoubleRangeField.newWithinQuery(FIELD_NAME, min, max);
+  }
+
+  /** Basic test */
+  public void testBasics() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+
+    // intersects (within)
+    Document document = new Document();
+    document.add(new DoubleRangeField(FIELD_NAME, new double[] {-10.0, -10.0}, new double[] {9.1, 10.1}));
+    writer.addDocument(document);
+
+    // intersects (crosses)
+    document = new Document();
+    document.add(new DoubleRangeField(FIELD_NAME, new double[] {10.0, -10.0}, new double[] {20.0, 10.0}));
+    writer.addDocument(document);
+
+    // intersects (contains)
+    document = new Document();
+    document.add(new DoubleRangeField(FIELD_NAME, new double[] {-20.0, -20.0}, new double[] {30.0, 30.1}));
+    writer.addDocument(document);
+
+    // intersects (crosses)
+    document = new Document();
+    document.add(new DoubleRangeField(FIELD_NAME, new double[] {-11.1, -11.2}, new double[] {1.23, 11.5}));
+    writer.addDocument(document);
+
+    // intersects (crosses)
+    document = new Document();
+    document.add(new DoubleRangeField(FIELD_NAME, new double[] {12.33, 1.2}, new double[] {15.1, 29.9}));
+    writer.addDocument(document);
+
+    // disjoint
+    document = new Document();
+    document.add(new DoubleRangeField(FIELD_NAME, new double[] {-122.33, 1.2}, new double[] {-115.1, 29.9}));
+    writer.addDocument(document);
+
+    // intersects (crosses)
+    document = new Document();
+    document.add(new DoubleRangeField(FIELD_NAME, new double[] {Double.NEGATIVE_INFINITY, 1.2}, new double[] {-11.0, 29.9}));
+    writer.addDocument(document);
+
+    // equal (within, contains, intersects)
+    document = new Document();
+    document.add(new DoubleRangeField(FIELD_NAME, new double[] {-11, -15}, new double[] {15, 20}));
+    writer.addDocument(document);
+
+    // search
+    IndexReader reader = writer.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+    assertEquals(7, searcher.count(DoubleRangeField.newIntersectsQuery(FIELD_NAME,
+        new double[] {-11.0, -15.0}, new double[] {15.0, 20.0})));
+    assertEquals(2, searcher.count(DoubleRangeField.newWithinQuery(FIELD_NAME,
+        new double[] {-11.0, -15.0}, new double[] {15.0, 20.0})));
+    assertEquals(2, searcher.count(DoubleRangeField.newContainsQuery(FIELD_NAME,
+        new double[] {-11.0, -15.0}, new double[] {15.0, 20.0})));
+
+    reader.close();
+    writer.close();
+    dir.close();
+  }
+}


[34/40] lucene-solr:apiv2: SOLR-8379: UI Cloud->Tree view now shows .txt files correctly. This closes #58

Posted by sa...@apache.org.
SOLR-8379: UI Cloud->Tree view now shows .txt files correctly. This closes #58


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

Branch: refs/heads/apiv2
Commit: 96280d15e3f37cd497ec6de7d9e2f099d151b2d6
Parents: d12b93e
Author: Jan H�ydahl <ja...@apache.org>
Authored: Fri Jul 29 00:44:53 2016 +0200
Committer: Jan H�ydahl <ja...@apache.org>
Committed: Fri Jul 29 09:33:44 2016 +0200

----------------------------------------------------------------------
 solr/CHANGES.txt                  | 2 ++
 solr/webapp/web/js/angular/app.js | 2 +-
 2 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/96280d15/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 58743b1..8e5b74f 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -180,6 +180,8 @@ Bug Fixes
 
 * SOLR-8645: managed-schema is now syntax highlighted in cloud->Tree view (Alexandre Rafalovitch via janhoy)
 
+* SOLR-8379: UI Cloud->Tree view now shows .txt files correctly (Alexandre Rafalovitch via janhoy)
+
 Optimizations
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/96280d15/solr/webapp/web/js/angular/app.js
----------------------------------------------------------------------
diff --git a/solr/webapp/web/js/angular/app.js b/solr/webapp/web/js/angular/app.js
index 8a1fe26..e7491fe 100644
--- a/solr/webapp/web/js/angular/app.js
+++ b/solr/webapp/web/js/angular/app.js
@@ -148,7 +148,7 @@ solrAdminApp.config([
 })
 .filter('highlight', function($sce) {
   return function(input, lang) {
-    if (lang && input && lang!="text") return hljs.highlight(lang, input).value;
+    if (lang && input && lang!="txt") return hljs.highlight(lang, input).value;
     return input;
   }
 })


[04/40] lucene-solr:apiv2: SOLR-9323: Expose ClusterSate.getCollectionStates which returns unverified list of collection names

Posted by sa...@apache.org.
SOLR-9323: Expose ClusterSate.getCollectionStates which returns unverified list of collection names


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

Branch: refs/heads/apiv2
Commit: 0ad365cbd069230bc638684b30bc4dc338e3a66d
Parents: f8a99dc
Author: Noble Paul <no...@apache.org>
Authored: Fri Jul 22 13:56:58 2016 +0530
Committer: Noble Paul <no...@apache.org>
Committed: Fri Jul 22 13:56:58 2016 +0530

----------------------------------------------------------------------
 .../org/apache/solr/handler/SQLHandler.java     |  4 ++--
 .../client/solrj/io/stream/CloudSolrStream.java | 25 ++++++++++----------
 .../client/solrj/io/stream/TopicStream.java     | 15 ++----------
 .../apache/solr/common/cloud/ClusterState.java  | 13 ++++++----
 4 files changed, 24 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0ad365cb/solr/core/src/java/org/apache/solr/handler/SQLHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/SQLHandler.java b/solr/core/src/java/org/apache/solr/handler/SQLHandler.java
index 4031858..f83c43f 100644
--- a/solr/core/src/java/org/apache/solr/handler/SQLHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/SQLHandler.java
@@ -1514,9 +1514,9 @@ public class SQLHandler extends RequestHandlerBase implements SolrCoreAware , Pe
       CloudSolrClient cloudSolrClient = this.context.getSolrClientCache().getCloudSolrClient(this.zkHost);
       cloudSolrClient.connect();
       ZkStateReader zkStateReader = cloudSolrClient.getZkStateReader();
-      Map<String, DocCollection> collections = zkStateReader.getClusterState().getCollectionsMap();
+      Set<String> collections = zkStateReader.getClusterState().getCollectionStates().keySet();
       if (collections.size() != 0) {
-        this.tables.addAll(collections.keySet());
+        this.tables.addAll(collections);
       }
       Collections.sort(this.tables);
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0ad365cb/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 8aba89c..2fb56ee 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
@@ -352,6 +352,16 @@ public class CloudSolrStream extends TupleStream implements Expressible {
     }
   }
 
+  public static Collection<Slice> getSlicesIgnoreCase(String name, ClusterState clusterState) {
+    for (String coll : clusterState.getCollectionStates().keySet()) {
+      if (coll.equalsIgnoreCase(name)) {
+        DocCollection collection = clusterState.getCollectionOrNull(coll);
+        if (collection != null) return collection.getActiveSlices();
+      }
+    }
+    return null;
+  }
+
   protected void constructStreams() throws IOException {
 
     try {
@@ -362,20 +372,9 @@ public class CloudSolrStream extends TupleStream implements Expressible {
       //System.out.println("Connected to zk an got cluster state.");
 
       Collection<Slice> slices = clusterState.getActiveSlices(this.collection);
-
+      if (slices == null) slices = getSlicesIgnoreCase(this.collection, clusterState);
       if (slices == null) {
-        //Try case insensitive match
-        Map<String, DocCollection> collectionsMap = clusterState.getCollectionsMap();
-        for (Map.Entry<String, DocCollection> entry : collectionsMap.entrySet()) {
-          if (entry.getKey().equalsIgnoreCase(collection)) {
-            slices = entry.getValue().getActiveSlices();
-            break;
-          }
-        }
-
-        if (slices == null) {
-          throw new Exception("Collection not found:" + this.collection);
-        }
+        throw new Exception("Collection not found:" + this.collection);
       }
 
       ModifiableSolrParams mParams = new ModifiableSolrParams(params); 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0ad365cb/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java
index c4343c6..c8bf610 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java
@@ -519,20 +519,9 @@ public class TopicStream extends CloudSolrStream implements Expressible  {
       //System.out.println("Connected to zk an got cluster state.");
 
       Collection<Slice> slices = clusterState.getActiveSlices(this.collection);
-
+      if (slices == null) slices = getSlicesIgnoreCase(this.collection, clusterState);
       if (slices == null) {
-        //Try case insensitive match
-        Map<String, DocCollection> collectionsMap = clusterState.getCollectionsMap();
-        for (Map.Entry<String, DocCollection> entry : collectionsMap.entrySet()) {
-          if (entry.getKey().equalsIgnoreCase(collection)) {
-            slices = entry.getValue().getActiveSlices();
-            break;
-          }
-        }
-
-        if (slices == null) {
-          throw new Exception("Collection not found:" + this.collection);
-        }
+        throw new Exception("Collection not found:" + this.collection);
       }
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0ad365cb/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java
index 55df271..3ab5a1f 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java
@@ -39,8 +39,8 @@ import org.noggit.JSONWriter;
 public class ClusterState implements JSONWriter.Writable {
   
   private final Integer znodeVersion;
-  
-  private final Map<String, CollectionRef> collectionStates;
+
+  private final Map<String, CollectionRef> collectionStates, immutableCollectionStates;
   private Set<String> liveNodes;
 
   /**
@@ -67,6 +67,7 @@ public class ClusterState implements JSONWriter.Writable {
     this.liveNodes = new HashSet<>(liveNodes.size());
     this.liveNodes.addAll(liveNodes);
     this.collectionStates = new LinkedHashMap<>(collectionStates);
+    this.immutableCollectionStates = Collections.unmodifiableMap(collectionStates);
   }
 
 
@@ -432,10 +433,12 @@ public class ClusterState implements JSONWriter.Writable {
     this.liveNodes = liveNodes;
   }
 
-  /**For internal use only
+  /** Be aware that this may return collections which may not exist now.
+   * You can confirm that this collection exists after verifying
+   * CollectionRef.get() != null
    */
-  Map<String, CollectionRef> getCollectionStates() {
-    return collectionStates;
+  public Map<String, CollectionRef> getCollectionStates() {
+    return immutableCollectionStates;
   }
 
   public static class CollectionRef {


[37/40] lucene-solr:apiv2: LUCENE-7390: another part of the revert

Posted by sa...@apache.org.
LUCENE-7390: another part of the revert


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

Branch: refs/heads/apiv2
Commit: 9554719f12080e233dbfe634f4a7066e138033ed
Parents: 1aecdd2
Author: Mike McCandless <mi...@apache.org>
Authored: Fri Jul 29 14:05:23 2016 -0400
Committer: Mike McCandless <mi...@apache.org>
Committed: Fri Jul 29 14:05:23 2016 -0400

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


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9554719f/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java b/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
index ce7e578..dcc7600 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
@@ -164,6 +164,6 @@ class PointValuesWriter {
       }
     };
 
-    writer.writeField(fieldInfo, reader, Math.max(indexWriterConfig.getRAMBufferSizeMB()/8.0));
+    writer.writeField(fieldInfo, reader);
   }
 }


[12/40] lucene-solr:apiv2: LUCENE-5438: make some APIs public so servers can actually use this feature

Posted by sa...@apache.org.
LUCENE-5438: make some APIs public so servers can actually use this feature


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

Branch: refs/heads/apiv2
Commit: 2d07ffd97fb4aec2a11aeddab40490044f3c2b49
Parents: 4c61d5a
Author: Mike McCandless <mi...@apache.org>
Authored: Mon Jul 25 06:00:11 2016 -0400
Committer: Mike McCandless <mi...@apache.org>
Committed: Mon Jul 25 06:00:37 2016 -0400

----------------------------------------------------------------------
 .../org/apache/lucene/replicator/nrt/Node.java   | 19 +++++++++++++++----
 .../lucene/replicator/nrt/PrimaryNode.java       |  5 +++++
 2 files changed, 20 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2d07ffd9/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/Node.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/Node.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/Node.java
index 759497a..aa66d9f 100644
--- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/Node.java
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/Node.java
@@ -25,6 +25,7 @@ import java.io.PrintStream;
 import java.nio.file.NoSuchFileException;
 import java.util.Locale;
 import java.util.Map;
+
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.DirectoryReader;
@@ -39,10 +40,10 @@ import org.apache.lucene.store.IndexInput;
  *
  * @lucene.experimental */
 
-abstract class Node implements Closeable {
+public abstract class Node implements Closeable {
 
-  static boolean VERBOSE_FILES = true;
-  static boolean VERBOSE_CONNECTIONS = false;
+  public static boolean VERBOSE_FILES = true;
+  public static boolean VERBOSE_CONNECTIONS = false;
 
   // Keys we store into IndexWriter's commit user data:
 
@@ -87,6 +88,16 @@ abstract class Node implements Closeable {
     this.printStream = printStream;
   }
 
+  /** Returns the {@link ReferenceManager} to use for acquiring and releasing searchers */
+  public ReferenceManager<IndexSearcher> getSearcherManager() {
+    return mgr;
+  }
+
+  /** Returns the {@link Directory} this node is writing to */
+  public Directory getDirectory() {
+    return dir;
+  }
+
   @Override
   public String toString() {
     return getClass().getSimpleName() + "(id=" + id + ")";
@@ -119,7 +130,7 @@ abstract class Node implements Closeable {
     }
   }
 
-  protected void message(String message) {
+  public void message(String message) {
     if (printStream != null) {
       long now = System.nanoTime();
       printStream.println(String.format(Locale.ROOT,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2d07ffd9/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java
index 1d04d08..8a3f07e 100644
--- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java
@@ -128,6 +128,11 @@ public abstract class PrimaryNode extends Node {
     }
   }
 
+  /** Returns the current primary generation, which is incremented each time a new primary is started for this index */
+  public long getPrimaryGen() {
+    return primaryGen;
+  }
+
   // TODO: in the future, we should separate "flush" (returns an incRef'd SegmentInfos) from "refresh" (open new NRT reader from
   // IndexWriter) so that the latter can be done concurrently while copying files out to replicas, minimizing the refresh time from the
   // replicas.  But fixing this is tricky because e.g. IndexWriter may complete a big merge just after returning the incRef'd SegmentInfos


[14/40] lucene-solr:apiv2: SOLR-9334: CloudSolrClient.collectionStateCache is unbounded

Posted by sa...@apache.org.
SOLR-9334: CloudSolrClient.collectionStateCache is unbounded


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

Branch: refs/heads/apiv2
Commit: 4ed68bc80e7990f4acd1b73dce3b5b8cd16d9fe5
Parents: dcc9a4b
Author: Noble Paul <no...@apache.org>
Authored: Mon Jul 25 20:47:19 2016 +0530
Committer: Noble Paul <no...@apache.org>
Committed: Mon Jul 25 20:47:19 2016 +0530

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  3 +++
 .../solr/client/solrj/impl/CloudSolrClient.java | 23 +++++++++++++++++++-
 2 files changed, 25 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4ed68bc8/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 55fae47..e392df3 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -161,6 +161,9 @@ Bug Fixes
 
 * SOLR-9309: Fix SolrCloud RTG response structure when multi ids requested but only 1 found (hossman)
 
+* SOLR-9334: CloudSolrClient.collectionStateCache is unbounded (noble)
+
+
 Optimizations
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4ed68bc8/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
index 876f7f8..b4c9b4f 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
@@ -38,6 +38,8 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.http.NoHttpResponseException;
 import org.apache.http.client.HttpClient;
@@ -147,10 +149,16 @@ public class CloudSolrClient extends SolrClient {
 
 
   protected final Map<String, ExpiringCachedDocCollection> collectionStateCache = new ConcurrentHashMap<String, ExpiringCachedDocCollection>(){
+    final Lock evictLock = new ReentrantLock(true);
     @Override
     public ExpiringCachedDocCollection get(Object key) {
       ExpiringCachedDocCollection val = super.get(key);
-      if(val == null) return null;
+      if(val == null) {
+        // a new collection is likely to be added now.
+        //check if there are stale items and remove them
+        evictStale();
+        return null;
+      }
       if(val.isExpired(timeToLive)) {
         super.remove(key);
         return null;
@@ -158,6 +166,19 @@ public class CloudSolrClient extends SolrClient {
       return val;
     }
 
+    void evictStale() {
+      if(!evictLock.tryLock()) return;
+      try {
+        for (Entry<String, ExpiringCachedDocCollection> e : entrySet()) {
+          if(e.getValue().isExpired(timeToLive)){
+            super.remove(e.getKey());
+          }
+        }
+      } finally {
+        evictLock.unlock();
+      }
+    }
+
   };
 
   class ExpiringCachedDocCollection {


[36/40] lucene-solr:apiv2: LUCENE-7390: revert this change, since it's obsoleted by the much better LUCENE-7396

Posted by sa...@apache.org.
LUCENE-7390: revert this change, since it's obsoleted by the much better LUCENE-7396


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

Branch: refs/heads/apiv2
Commit: 1aecdd28d130c757770de67bfde52f3c989bd134
Parents: 60975d2
Author: Mike McCandless <mi...@apache.org>
Authored: Fri Jul 29 14:02:00 2016 -0400
Committer: Mike McCandless <mi...@apache.org>
Committed: Fri Jul 29 14:02:00 2016 -0400

----------------------------------------------------------------------
 lucene/CHANGES.txt                               |  5 -----
 .../simpletext/SimpleTextPointsWriter.java       |  4 ++--
 .../org/apache/lucene/codecs/PointsWriter.java   | 11 +++--------
 .../codecs/lucene60/Lucene60PointsWriter.java    | 15 +++++++--------
 .../lucene/index/DocumentsWriterPerThread.java   |  2 +-
 .../org/apache/lucene/index/IndexWriter.java     |  2 +-
 .../lucene/index/LiveIndexWriterConfig.java      | 19 +++++++++++--------
 .../apache/lucene/index/PointValuesWriter.java   |  5 +----
 .../org/apache/lucene/util/bkd/BKDWriter.java    |  2 +-
 .../lucene60/TestLucene60PointsFormat.java       |  5 +++--
 .../apache/lucene/search/TestPointQueries.java   |  5 +++--
 .../apache/lucene/spatial3d/TestGeo3DPoint.java  |  5 +++--
 .../codecs/asserting/AssertingPointsFormat.java  |  4 ++--
 .../lucene/codecs/cranky/CrankyPointsFormat.java |  4 ++--
 .../apache/lucene/geo/BaseGeoPointTestCase.java  |  3 ++-
 .../org/apache/lucene/index/RandomCodec.java     |  9 ++++++---
 16 files changed, 48 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1aecdd28/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 67e2683..45bc830 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -138,11 +138,6 @@ Improvements
 
 * LUCENE-7385: Improve/fix assert messages in SpanScorer. (David Smiley)
 
-* LUCENE-7390: Improve performance of indexing points by allowing the
-  codec to use transient heap in proportion to IndexWriter's RAM
-  buffer, instead of a fixed 16.0 MB.  A custom codec can still
-  override the buffer size itself. (Mike McCandless)
-
 * LUCENE-7393: Add ICUTokenizer option to parse Myanmar text as syllables instead of words,
   because the ICU word-breaking algorithm has some issues. This allows for the previous 
   tokenization used before Lucene 5. (AM, Robert Muir)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1aecdd28/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsWriter.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsWriter.java
index 67289b6..8d5c034 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsWriter.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsWriter.java
@@ -68,7 +68,7 @@ class SimpleTextPointsWriter extends PointsWriter {
   }
 
   @Override
-  public void writeField(FieldInfo fieldInfo, PointsReader values, double maxMBSortInHeap) throws IOException {
+  public void writeField(FieldInfo fieldInfo, PointsReader values) throws IOException {
 
     boolean singleValuePerDoc = values.size(fieldInfo.name) == values.getDocCount(fieldInfo.name);
 
@@ -79,7 +79,7 @@ class SimpleTextPointsWriter extends PointsWriter {
                                           fieldInfo.getPointDimensionCount(),
                                           fieldInfo.getPointNumBytes(),
                                           BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE,
-                                          maxMBSortInHeap,
+                                          BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP,
                                           values.size(fieldInfo.name),
                                           singleValuePerDoc) {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1aecdd28/lucene/core/src/java/org/apache/lucene/codecs/PointsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/PointsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/PointsWriter.java
index 8458497..05084db 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/PointsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/PointsWriter.java
@@ -22,7 +22,6 @@ import java.io.IOException;
 
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.MergeState;
-import org.apache.lucene.util.bkd.BKDWriter;
 
 /** Abstract API to write points
  *
@@ -35,9 +34,8 @@ public abstract class PointsWriter implements Closeable {
   protected PointsWriter() {
   }
 
-  /** Write all values contained in the provided reader.  {@code maxMBSortInHeap} is the maximum
-   *  transient heap that can be used to sort values, before spilling to disk for offline sorting */
-  public abstract void writeField(FieldInfo fieldInfo, PointsReader values, double maxMBSortInHeap) throws IOException;
+  /** Write all values contained in the provided reader */
+  public abstract void writeField(FieldInfo fieldInfo, PointsReader values) throws IOException;
 
   /** Default naive merge implementation for one field: it just re-indexes all the values
    *  from the incoming segment.  The default codec overrides this for 1D fields and uses
@@ -147,10 +145,7 @@ public abstract class PointsWriter implements Closeable {
                  public int getDocCount(String fieldName) {
                    return finalDocCount;
                  }
-               },
-               // TODO: also let merging of > 1D fields tap into IW's indexing buffer size, somehow (1D fields do an optimized merge sort
-               // and don't need heap)
-               BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP);
+               });
   }
 
   /** Default merge implementation to merge incoming points readers by visiting all their points and

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1aecdd28/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java
index 5fedf64..ff9de58 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java
@@ -40,9 +40,7 @@ import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.bkd.BKDReader;
 import org.apache.lucene.util.bkd.BKDWriter;
 
-/** Writes dimensional values
- *
- * @lucene.experimental */
+/** Writes dimensional values */
 public class Lucene60PointsWriter extends PointsWriter implements Closeable {
 
   /** Output used to write the BKD tree data file */
@@ -53,13 +51,15 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
 
   final SegmentWriteState writeState;
   final int maxPointsInLeafNode;
+  final double maxMBSortInHeap;
   private boolean finished;
 
   /** Full constructor */
-  public Lucene60PointsWriter(SegmentWriteState writeState, int maxPointsInLeafNode) throws IOException {
+  public Lucene60PointsWriter(SegmentWriteState writeState, int maxPointsInLeafNode, double maxMBSortInHeap) throws IOException {
     assert writeState.fieldInfos.hasPointValues();
     this.writeState = writeState;
     this.maxPointsInLeafNode = maxPointsInLeafNode;
+    this.maxMBSortInHeap = maxMBSortInHeap;
     String dataFileName = IndexFileNames.segmentFileName(writeState.segmentInfo.name,
                                                          writeState.segmentSuffix,
                                                          Lucene60PointsFormat.DATA_EXTENSION);
@@ -81,11 +81,11 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
 
   /** Uses the defaults values for {@code maxPointsInLeafNode} (1024) and {@code maxMBSortInHeap} (16.0) */
   public Lucene60PointsWriter(SegmentWriteState writeState) throws IOException {
-    this(writeState, BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE);
+    this(writeState, BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE, BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP);
   }
 
   @Override
-  public void writeField(FieldInfo fieldInfo, PointsReader values, double maxMBSortInHeap) throws IOException {
+  public void writeField(FieldInfo fieldInfo, PointsReader values) throws IOException {
 
     boolean singleValuePerDoc = values.size(fieldInfo.name) == values.getDocCount(fieldInfo.name);
 
@@ -182,8 +182,7 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
                                                 fieldInfo.getPointDimensionCount(),
                                                 fieldInfo.getPointNumBytes(),
                                                 maxPointsInLeafNode,
-                                                // NOTE: not used, since BKDWriter.merge does a merge sort:
-                                                BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP,
+                                                maxMBSortInHeap,
                                                 totMaxSize,
                                                 singleValuePerDoc)) {
             List<BKDReader> bkdReaders = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1aecdd28/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 351235e..e72145c 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
@@ -153,7 +153,7 @@ class DocumentsWriterPerThread {
   final Allocator byteBlockAllocator;
   final IntBlockPool.Allocator intBlockAllocator;
   private final AtomicLong pendingNumDocs;
-  final LiveIndexWriterConfig indexWriterConfig;
+  private final LiveIndexWriterConfig indexWriterConfig;
   private final boolean enableTestPoints;
   private final IndexWriter indexWriter;
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1aecdd28/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
index bdd9552..0fb23d9 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -762,7 +762,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
    * {@link #getConfig()}.
    * 
    * <p>
-   * <b>NOTE:</b> after this writer is created, the given configuration instance
+   * <b>NOTE:</b> after ths writer is created, the given configuration instance
    * cannot be passed to another writer.
    * 
    * @param d

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1aecdd28/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java b/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java
index fe4924d..cec70c0 100644
--- a/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java
+++ b/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java
@@ -168,14 +168,9 @@ public class LiveIndexWriterConfig {
   
   /**
    * Determines the amount of RAM that may be used for buffering added documents
-   * and deletions before beginning to flush them to the Directory.  For
-   * faster indexing performance it's best to use as large a RAM buffer as you can.
-   * <p>
-   * Note that this setting is not a hard limit on memory usage during indexing, as
-   * transient and non-trivial memory well beyond this buffer size may be used,
-   * for example due to segment merges or writing points to new segments.
-   * For application stability the available memory in the JVM
-   * should be significantly larger than the RAM buffer used for indexing.
+   * and deletions before they are flushed to the Directory. Generally for
+   * faster indexing performance it's best to flush by RAM usage instead of
+   * document count and use as large a RAM buffer as you can.
    * <p>
    * When this is set, the writer will flush whenever buffered documents and
    * deletions use this much RAM. Pass in
@@ -183,6 +178,14 @@ public class LiveIndexWriterConfig {
    * due to RAM usage. Note that if flushing by document count is also enabled,
    * then the flush will be triggered by whichever comes first.
    * <p>
+   * The maximum RAM limit is inherently determined by the JVMs available
+   * memory. Yet, an {@link IndexWriter} session can consume a significantly
+   * larger amount of memory than the given RAM limit since this limit is just
+   * an indicator when to flush memory resident documents to the Directory.
+   * Flushes are likely happen concurrently while other threads adding documents
+   * to the writer. For application stability the available memory in the JVM
+   * should be significantly larger than the RAM buffer used for indexing.
+   * <p>
    * <b>NOTE</b>: the account of RAM usage for pending deletions is only
    * approximate. Specifically, if you delete by Query, Lucene currently has no
    * way to measure the RAM usage of individual Queries so the accounting will

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1aecdd28/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java b/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
index b4decb6..ce7e578 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
@@ -25,7 +25,6 @@ import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.Counter;
-import org.apache.lucene.util.bkd.BKDWriter;
 
 /** Buffers up pending byte[][] value(s) per doc, then flushes when segment flushes. */
 class PointValuesWriter {
@@ -37,7 +36,6 @@ class PointValuesWriter {
   private int numDocs;
   private int lastDocID = -1;
   private final int packedBytesLength;
-  private final LiveIndexWriterConfig indexWriterConfig;
 
   public PointValuesWriter(DocumentsWriterPerThread docWriter, FieldInfo fieldInfo) {
     this.fieldInfo = fieldInfo;
@@ -46,7 +44,6 @@ class PointValuesWriter {
     docIDs = new int[16];
     iwBytesUsed.addAndGet(16 * Integer.BYTES);
     packedBytesLength = fieldInfo.getPointDimensionCount() * fieldInfo.getPointNumBytes();
-    indexWriterConfig = docWriter.indexWriterConfig;
   }
 
   // TODO: if exactly the same value is added to exactly the same doc, should we dedup?
@@ -167,6 +164,6 @@ class PointValuesWriter {
       }
     };
 
-    writer.writeField(fieldInfo, reader, Math.max(indexWriterConfig.getRAMBufferSizeMB()/8.0, BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP));
+    writer.writeField(fieldInfo, reader, Math.max(indexWriterConfig.getRAMBufferSizeMB()/8.0));
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1aecdd28/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
index d0d7dca..8bd66d0 100644
--- a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
@@ -205,7 +205,7 @@ public class BKDWriter implements Closeable {
     // all recursive halves (i.e. 16 + 8 + 4 + 2) so the memory usage is 2X
     // what that level would consume, so we multiply by 0.5 to convert from
     // bytes to points here.  Each dimension has its own sorted partition, so
-    // we must divide by numDims as well.
+    // we must divide by numDims as wel.
 
     maxPointsSortInHeap = (int) (0.5 * (maxMBSortInHeap * 1024 * 1024) / (bytesPerDoc * numDims));
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1aecdd28/lucene/core/src/test/org/apache/lucene/codecs/lucene60/TestLucene60PointsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene60/TestLucene60PointsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene60/TestLucene60PointsFormat.java
index 4b898c3..afa8ec4 100644
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene60/TestLucene60PointsFormat.java
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene60/TestLucene60PointsFormat.java
@@ -41,8 +41,9 @@ public class TestLucene60PointsFormat extends BasePointsFormatTestCase {
     if (random().nextBoolean()) {
       // randomize parameters
       int maxPointsInLeafNode = TestUtil.nextInt(random(), 50, 500);
+      double maxMBSortInHeap = 3.0 + (3*random().nextDouble());
       if (VERBOSE) {
-        System.out.println("TEST: using Lucene60PointsFormat with maxPointsInLeafNode=" + maxPointsInLeafNode);
+        System.out.println("TEST: using Lucene60PointsFormat with maxPointsInLeafNode=" + maxPointsInLeafNode + " and maxMBSortInHeap=" + maxMBSortInHeap);
       }
 
       // sneaky impersonation!
@@ -52,7 +53,7 @@ public class TestLucene60PointsFormat extends BasePointsFormatTestCase {
           return new PointsFormat() {
             @Override
             public PointsWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
-              return new Lucene60PointsWriter(writeState, maxPointsInLeafNode);
+              return new Lucene60PointsWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap);
             }
 
             @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1aecdd28/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java b/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
index c139b64..cf8372d 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
@@ -1156,8 +1156,9 @@ public class TestPointQueries extends LuceneTestCase {
   private static Codec getCodec() {
     if (Codec.getDefault().getName().equals("Lucene62")) {
       int maxPointsInLeafNode = TestUtil.nextInt(random(), 16, 2048);
+      double maxMBSortInHeap = 5.0 + (3*random().nextDouble());
       if (VERBOSE) {
-        System.out.println("TEST: using Lucene60PointsFormat with maxPointsInLeafNode=" + maxPointsInLeafNode);
+        System.out.println("TEST: using Lucene60PointsFormat with maxPointsInLeafNode=" + maxPointsInLeafNode + " and maxMBSortInHeap=" + maxMBSortInHeap);
       }
 
       return new FilterCodec("Lucene62", Codec.getDefault()) {
@@ -1166,7 +1167,7 @@ public class TestPointQueries extends LuceneTestCase {
           return new PointsFormat() {
             @Override
             public PointsWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
-              return new Lucene60PointsWriter(writeState, maxPointsInLeafNode);
+              return new Lucene60PointsWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap);
             }
 
             @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1aecdd28/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/TestGeo3DPoint.java
----------------------------------------------------------------------
diff --git a/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/TestGeo3DPoint.java b/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/TestGeo3DPoint.java
index bb98145..d9baf61 100644
--- a/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/TestGeo3DPoint.java
+++ b/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/TestGeo3DPoint.java
@@ -87,8 +87,9 @@ public class TestGeo3DPoint extends LuceneTestCase {
   private static Codec getCodec() {
     if (Codec.getDefault().getName().equals("Lucene62")) {
       int maxPointsInLeafNode = TestUtil.nextInt(random(), 16, 2048);
+      double maxMBSortInHeap = 3.0 + (3*random().nextDouble());
       if (VERBOSE) {
-        System.out.println("TEST: using Lucene60PointsFormat with maxPointsInLeafNode=" + maxPointsInLeafNode);
+        System.out.println("TEST: using Lucene60PointsFormat with maxPointsInLeafNode=" + maxPointsInLeafNode + " and maxMBSortInHeap=" + maxMBSortInHeap);
       }
 
       return new FilterCodec("Lucene62", Codec.getDefault()) {
@@ -97,7 +98,7 @@ public class TestGeo3DPoint extends LuceneTestCase {
           return new PointsFormat() {
             @Override
             public PointsWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
-              return new Lucene60PointsWriter(writeState, maxPointsInLeafNode);
+              return new Lucene60PointsWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap);
             }
 
             @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1aecdd28/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java
index 0bbf2c6..c6f5485 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java
@@ -255,11 +255,11 @@ public final class AssertingPointsFormat extends PointsFormat {
     }
     
     @Override
-    public void writeField(FieldInfo fieldInfo, PointsReader values, double maxMBSortInHeap) throws IOException {
+    public void writeField(FieldInfo fieldInfo, PointsReader values) throws IOException {
       if (fieldInfo.getPointDimensionCount() == 0) {
         throw new IllegalArgumentException("writing field=\"" + fieldInfo.name + "\" but pointDimensionalCount is 0");
       }
-      in.writeField(fieldInfo, values, maxMBSortInHeap);
+      in.writeField(fieldInfo, values);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1aecdd28/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyPointsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyPointsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyPointsFormat.java
index ffd9a8c..fd2260b 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyPointsFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyPointsFormat.java
@@ -56,11 +56,11 @@ class CrankyPointsFormat extends PointsFormat {
     }
 
     @Override
-    public void writeField(FieldInfo fieldInfo, PointsReader values, double maxMBSortInHeap) throws IOException {
+    public void writeField(FieldInfo fieldInfo, PointsReader values) throws IOException {
       if (random.nextInt(100) == 0) {
         throw new IOException("Fake IOException");
       }  
-      delegate.writeField(fieldInfo, values, maxMBSortInHeap);
+      delegate.writeField(fieldInfo, values);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1aecdd28/lucene/test-framework/src/java/org/apache/lucene/geo/BaseGeoPointTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/geo/BaseGeoPointTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/geo/BaseGeoPointTestCase.java
index 926132f..275c186 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/geo/BaseGeoPointTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/geo/BaseGeoPointTestCase.java
@@ -67,6 +67,7 @@ import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.SloppyMath;
 import org.apache.lucene.util.TestUtil;
+import org.apache.lucene.util.bkd.BKDWriter;
 
 /**
  * Abstract class to do basic tests for a geospatial impl (high level
@@ -1247,7 +1248,7 @@ public abstract class BaseGeoPointTestCase extends LuceneTestCase {
         return new PointsFormat() {
           @Override
           public PointsWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
-            return new Lucene60PointsWriter(writeState, pointsInLeaf);
+            return new Lucene60PointsWriter(writeState, pointsInLeaf, BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP);
           }
   
           @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1aecdd28/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java b/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
index 6e8f7bc..127549f 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
@@ -92,6 +92,7 @@ public class RandomCodec extends AssertingCodec {
   // which is less effective for testing.
   // TODO: improve how we randomize this...
   private final int maxPointsInLeafNode;
+  private final double maxMBSortInHeap;
   private final int bkdSplitRandomSeed;
 
   @Override
@@ -102,9 +103,9 @@ public class RandomCodec extends AssertingCodec {
 
         // Randomize how BKDWriter chooses its splis:
 
-        return new Lucene60PointsWriter(writeState, maxPointsInLeafNode) {
+        return new Lucene60PointsWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap) {
           @Override
-          public void writeField(FieldInfo fieldInfo, PointsReader values, double maxMBSortInHeap) throws IOException {
+          public void writeField(FieldInfo fieldInfo, PointsReader values) throws IOException {
 
             boolean singleValuePerDoc = values.size(fieldInfo.name) == values.getDocCount(fieldInfo.name);
 
@@ -184,6 +185,7 @@ public class RandomCodec extends AssertingCodec {
     int lowFreqCutoff = TestUtil.nextInt(random, 2, 100);
 
     maxPointsInLeafNode = TestUtil.nextInt(random, 16, 2048);
+    maxMBSortInHeap = 5.0 + (3*random.nextDouble());
     bkdSplitRandomSeed = random.nextInt();
 
     add(avoidCodecs,
@@ -251,7 +253,8 @@ public class RandomCodec extends AssertingCodec {
   public String toString() {
     return super.toString() + ": " + previousMappings.toString() +
            ", docValues:" + previousDVMappings.toString() +
-           ", maxPointsInLeafNode=" + maxPointsInLeafNode;
+           ", maxPointsInLeafNode=" + maxPointsInLeafNode +
+           ", maxMBSortInHeap=" + maxMBSortInHeap;
   }
 
   /** Just like {@link BKDWriter} except it evilly picks random ways to split cells on


[31/40] lucene-solr:apiv2: SOLR-8645: managed-schema is now syntax highlighted in cloud->Tree view. This closes #57

Posted by sa...@apache.org.
SOLR-8645: managed-schema is now syntax highlighted in cloud->Tree view. This closes #57


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

Branch: refs/heads/apiv2
Commit: 1fb4c5d46d6f787a43121d896e64825252ea3b62
Parents: b7aa0b5
Author: Jan H�ydahl <ja...@apache.org>
Authored: Fri Jul 29 00:03:24 2016 +0200
Committer: Jan H�ydahl <ja...@apache.org>
Committed: Fri Jul 29 00:03:24 2016 +0200

----------------------------------------------------------------------
 solr/CHANGES.txt                                | 2 ++
 solr/webapp/web/js/angular/controllers/cloud.js | 5 +++++
 2 files changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1fb4c5d4/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 3011cec..22d4e99 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -175,6 +175,8 @@ Bug Fixes
 
 * SOLR-8596: Web UI doesn't correctly generate queries which include local parameters (Alexandre Rafalovitch, janhoy)
 
+* SOLR-8645: managed-schema is now syntax highlighted in cloud->Tree view (Alexandre Rafalovitch via janhoy)
+
 Optimizations
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1fb4c5d4/solr/webapp/web/js/angular/controllers/cloud.js
----------------------------------------------------------------------
diff --git a/solr/webapp/web/js/angular/controllers/cloud.js b/solr/webapp/web/js/angular/controllers/cloud.js
index c150c5a..2d0dae0 100644
--- a/solr/webapp/web/js/angular/controllers/cloud.js
+++ b/solr/webapp/web/js/angular/controllers/cloud.js
@@ -55,6 +55,11 @@ var treeSubController = function($scope, Zookeeper) {
             var path = data.znode.path.split( '.' );
             if(path.length >1) {
               $scope.lang = path.pop();
+            } else {
+              var lastPathElement = data.znode.path.split( '/' ).pop();
+              if (lastPathElement == "managed-schema") {
+                  $scope.lang = "xml";
+              }
             }
             $scope.showData = true;
         });


[28/40] lucene-solr:apiv2: SOLR-9200: Add Delegation Token Support to Solr

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/core/src/test/org/apache/solr/cloud/TestSolrCloudWithDelegationTokens.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestSolrCloudWithDelegationTokens.java b/solr/core/src/test/org/apache/solr/cloud/TestSolrCloudWithDelegationTokens.java
new file mode 100644
index 0000000..ae1c439
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/TestSolrCloudWithDelegationTokens.java
@@ -0,0 +1,405 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.cloud;
+
+import junit.framework.Assert;
+import org.apache.hadoop.util.Time;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.DelegationTokenRequest;
+import org.apache.solr.client.solrj.response.DelegationTokenResponse;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import static org.apache.solr.security.HttpParamDelegationTokenAuthenticationHandler.USER_PARAM;
+
+import org.apache.http.HttpStatus;
+import org.apache.solr.security.HttpParamDelegationTokenAuthenticationHandler;
+import org.apache.solr.security.KerberosPlugin;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.lang.invoke.MethodHandles;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test the delegation token support in the {@link org.apache.solr.security.KerberosPlugin}.
+ */
+@LuceneTestCase.Slow
+public class TestSolrCloudWithDelegationTokens extends SolrTestCaseJ4 {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  private static final int NUM_SERVERS = 2;
+  private static MiniSolrCloudCluster miniCluster;
+  private static HttpSolrClient solrClientPrimary;
+  private static HttpSolrClient solrClientSecondary;
+
+  @BeforeClass
+  public static void startup() throws Exception {
+    System.setProperty("authenticationPlugin", KerberosPlugin.class.getName());
+    System.setProperty(KerberosPlugin.DELEGATION_TOKEN_ENABLED, "true");
+    System.setProperty(KerberosPlugin.AUTH_HANDLER_PARAM,
+        HttpParamDelegationTokenAuthenticationHandler.class.getName());
+    System.setProperty("solr.kerberos.cookie.domain", "127.0.0.1");
+
+    miniCluster = new MiniSolrCloudCluster(NUM_SERVERS, createTempDir(), buildJettyConfig("/solr"));
+    JettySolrRunner runnerPrimary = miniCluster.getJettySolrRunners().get(0);
+    solrClientPrimary =
+        new HttpSolrClient.Builder(runnerPrimary.getBaseUrl().toString())
+            .build();
+    JettySolrRunner runnerSecondary = miniCluster.getJettySolrRunners().get(1);
+    solrClientSecondary =
+        new HttpSolrClient.Builder(runnerSecondary.getBaseUrl().toString())
+            .build();
+  }
+
+  @AfterClass
+  public static void shutdown() throws Exception {
+    if (miniCluster != null) {
+      miniCluster.shutdown();
+    }
+    miniCluster = null;
+    solrClientPrimary.close();
+    solrClientPrimary = null;
+    solrClientSecondary.close();
+    solrClientSecondary = null;
+    System.clearProperty("authenticationPlugin");
+    System.clearProperty(KerberosPlugin.DELEGATION_TOKEN_ENABLED);
+    System.clearProperty(KerberosPlugin.AUTH_HANDLER_PARAM);
+    System.clearProperty("solr.kerberos.cookie.domain");
+  }
+
+  private String getDelegationToken(final String renewer, final String user, HttpSolrClient solrClient) throws Exception {
+    DelegationTokenRequest.Get get = new DelegationTokenRequest.Get(renewer) {
+      @Override
+      public SolrParams getParams() {
+        ModifiableSolrParams params = new ModifiableSolrParams(super.getParams());
+        params.set(USER_PARAM, user);
+        return params;
+      }
+    };
+    DelegationTokenResponse.Get getResponse = get.process(solrClient);
+    return getResponse.getDelegationToken();
+  }
+
+  private long renewDelegationToken(final String token, final int expectedStatusCode,
+      final String user, HttpSolrClient client) throws Exception {
+    DelegationTokenRequest.Renew renew = new DelegationTokenRequest.Renew(token) {
+      @Override
+      public SolrParams getParams() {
+        ModifiableSolrParams params = new ModifiableSolrParams(super.getParams());
+        params.set(USER_PARAM, user);
+        return params;
+      }
+
+      @Override
+      public Set<String> getQueryParams() {
+        Set<String> queryParams = super.getQueryParams();
+        queryParams.add(USER_PARAM);
+        return queryParams;
+      }
+    };
+    try {
+      DelegationTokenResponse.Renew renewResponse = renew.process(client);
+      assertEquals(HttpStatus.SC_OK, expectedStatusCode);
+      return renewResponse.getExpirationTime();
+    } catch (HttpSolrClient.RemoteSolrException ex) {
+      assertEquals(expectedStatusCode, ex.code());
+      return -1;
+    }
+  }
+
+  private void cancelDelegationToken(String token, int expectedStatusCode, HttpSolrClient client)
+  throws Exception {
+    DelegationTokenRequest.Cancel cancel = new DelegationTokenRequest.Cancel(token);
+    try {
+      cancel.process(client);
+      assertEquals(HttpStatus.SC_OK, expectedStatusCode);
+    } catch (HttpSolrClient.RemoteSolrException ex) {
+      assertEquals(expectedStatusCode, ex.code());
+    }
+  }
+
+  private void doSolrRequest(String token, int expectedStatusCode, HttpSolrClient client)
+  throws Exception {
+    doSolrRequest(token, expectedStatusCode, client, 1);
+  }
+
+  private void doSolrRequest(String token, int expectedStatusCode, HttpSolrClient client, int trials)
+  throws Exception {
+    int lastStatusCode = 0;
+    for (int i = 0; i < trials; ++i) {
+      lastStatusCode = getStatusCode(token, null, null, client);
+      if (lastStatusCode == expectedStatusCode) {
+        return;
+      }
+      Thread.sleep(1000);
+    }
+    assertEquals("Did not receieve excepted status code", expectedStatusCode, lastStatusCode);
+  }
+
+  private SolrRequest getAdminRequest(final SolrParams params) {
+    return new CollectionAdminRequest.List() {
+      @Override
+      public SolrParams getParams() {
+        ModifiableSolrParams p = new ModifiableSolrParams(super.getParams());
+        p.add(params);
+        return p;
+      }
+    };
+  }
+
+  private int getStatusCode(String token, final String user, final String op, HttpSolrClient client)
+  throws Exception {
+    HttpSolrClient delegationTokenServer =
+        new HttpSolrClient.Builder(client.getBaseURL().toString())
+            .withDelegationToken(token)
+            .withResponseParser(client.getParser())
+            .build();
+    try {
+      ModifiableSolrParams p = new ModifiableSolrParams();
+      if (user != null) p.set(USER_PARAM, user);
+      if (op != null) p.set("op", op);
+      SolrRequest req = getAdminRequest(p);
+      if (user != null || op != null) {
+        Set<String> queryParams = new HashSet<String>();
+        if (user != null) queryParams.add(USER_PARAM);
+        if (op != null) queryParams.add("op");
+        req.setQueryParams(queryParams);
+      }
+      try {
+        delegationTokenServer.request(req, null, null);
+        return HttpStatus.SC_OK;
+      } catch (HttpSolrClient.RemoteSolrException re) {
+        return re.code();
+      }
+    } finally {
+      delegationTokenServer.close();
+    }
+  }
+
+  private void doSolrRequest(HttpSolrClient client, SolrRequest request,
+      int expectedStatusCode) throws Exception {
+    try {
+      client.request(request);
+      assertEquals(HttpStatus.SC_OK, expectedStatusCode);
+    } catch (HttpSolrClient.RemoteSolrException ex) {
+      assertEquals(expectedStatusCode, ex.code());
+    }
+  }
+
+  private void verifyTokenValid(String token) throws Exception {
+     // pass with token
+    doSolrRequest(token, HttpStatus.SC_OK, solrClientPrimary);
+
+    // fail without token
+    doSolrRequest(null, ErrorCode.UNAUTHORIZED.code, solrClientPrimary);
+
+    // pass with token on other server
+    doSolrRequest(token, HttpStatus.SC_OK, solrClientSecondary);
+
+    // fail without token on other server
+    doSolrRequest(null, ErrorCode.UNAUTHORIZED.code, solrClientSecondary);
+  }
+
+  /**
+   * Test basic Delegation Token get/verify
+   */
+  @Test
+  public void testDelegationTokenVerify() throws Exception {
+    final String user = "bar";
+
+    // Get token
+    String token = getDelegationToken(null, user, solrClientPrimary);
+    assertNotNull(token);
+    verifyTokenValid(token);
+  }
+
+  private void verifyTokenCancelled(String token, HttpSolrClient client) throws Exception {
+    // fail with token on both servers.  If cancelToOtherURL is true,
+    // the request went to other url, so FORBIDDEN should be returned immediately.
+    // The cancelled token may take awhile to propogate to the standard url (via ZK).
+    // This is of course the opposite if cancelToOtherURL is false.
+    doSolrRequest(token, ErrorCode.FORBIDDEN.code, client, 10);
+
+    // fail without token on both servers
+    doSolrRequest(null, ErrorCode.UNAUTHORIZED.code, solrClientPrimary);
+    doSolrRequest(null, ErrorCode.UNAUTHORIZED.code, solrClientSecondary);
+  }
+
+  @Test
+  public void testDelegationTokenCancel() throws Exception {
+    {
+      // Get token
+      String token = getDelegationToken(null, "user", solrClientPrimary);
+      assertNotNull(token);
+
+      // cancel token, note don't need to be authenticated to cancel (no user specified)
+      cancelDelegationToken(token, HttpStatus.SC_OK, solrClientPrimary);
+      verifyTokenCancelled(token, solrClientPrimary);
+    }
+
+    {
+      // cancel token on different server from where we got it
+      String token = getDelegationToken(null, "user", solrClientPrimary);
+      assertNotNull(token);
+
+      cancelDelegationToken(token, HttpStatus.SC_OK, solrClientSecondary);
+      verifyTokenCancelled(token, solrClientSecondary);
+    }
+  }
+
+  @Test
+  public void testDelegationTokenCancelFail() throws Exception {
+    // cancel a bogus token
+    cancelDelegationToken("BOGUS", ErrorCode.NOT_FOUND.code, solrClientPrimary);
+
+    {
+      // cancel twice, first on same server
+      String token = getDelegationToken(null, "bar", solrClientPrimary);
+      assertNotNull(token);
+      cancelDelegationToken(token, HttpStatus.SC_OK, solrClientPrimary);
+      cancelDelegationToken(token, ErrorCode.NOT_FOUND.code, solrClientSecondary);
+      cancelDelegationToken(token, ErrorCode.NOT_FOUND.code, solrClientPrimary);
+    }
+
+    {
+      // cancel twice, first on other server
+      String token = getDelegationToken(null, "bar", solrClientPrimary);
+      assertNotNull(token);
+      cancelDelegationToken(token, HttpStatus.SC_OK, solrClientSecondary);
+      cancelDelegationToken(token, ErrorCode.NOT_FOUND.code, solrClientSecondary);
+      cancelDelegationToken(token, ErrorCode.NOT_FOUND.code, solrClientPrimary);
+    }
+  }
+
+  private void verifyDelegationTokenRenew(String renewer, String user)
+  throws Exception {
+    {
+      // renew on same server
+      String token = getDelegationToken(renewer, user, solrClientPrimary);
+      assertNotNull(token);
+      long now = Time.now();
+      assertTrue(renewDelegationToken(token, HttpStatus.SC_OK, user, solrClientPrimary) > now);
+      verifyTokenValid(token);
+    }
+
+    {
+      // renew on different server
+      String token = getDelegationToken(renewer, user, solrClientPrimary);
+      assertNotNull(token);
+      long now = Time.now();
+      assertTrue(renewDelegationToken(token, HttpStatus.SC_OK, user, solrClientSecondary) > now);
+      verifyTokenValid(token);
+    }
+  }
+
+  @Test
+  public void testDelegationTokenRenew() throws Exception {
+    // test with specifying renewer
+    verifyDelegationTokenRenew("bar", "bar");
+
+    // test without specifying renewer
+    verifyDelegationTokenRenew(null, "bar");
+  }
+
+  @Test
+  public void testDelegationTokenRenewFail() throws Exception {
+    // don't set renewer and try to renew as an a different user
+    String token = getDelegationToken(null, "bar", solrClientPrimary);
+    assertNotNull(token);
+    renewDelegationToken(token, ErrorCode.FORBIDDEN.code, "foo", solrClientPrimary);
+    renewDelegationToken(token, ErrorCode.FORBIDDEN.code, "foo", solrClientSecondary);
+
+    // set renewer and try to renew as different user
+    token = getDelegationToken("renewUser", "bar", solrClientPrimary);
+    assertNotNull(token);
+    renewDelegationToken(token, ErrorCode.FORBIDDEN.code, "notRenewUser", solrClientPrimary);
+    renewDelegationToken(token, ErrorCode.FORBIDDEN.code, "notRenewUser", solrClientSecondary);
+  }
+
+  /**
+   * Test that a non-delegation-token "op" http param is handled correctly
+   */
+  @Test
+  public void testDelegationOtherOp() throws Exception {
+    assertEquals(HttpStatus.SC_OK, getStatusCode(null, "bar", "someSolrOperation", solrClientPrimary));
+  }
+
+  @Test
+  public void testZNodePaths() throws Exception {
+    getDelegationToken(null, "bar", solrClientPrimary);
+    SolrZkClient zkClient = new SolrZkClient(miniCluster.getZkServer().getZkAddress(), 1000);
+    try {
+      assertTrue(zkClient.exists("/security/zkdtsm", true));
+      assertTrue(zkClient.exists("/security/token", true));
+    } finally {
+      zkClient.close();
+    }
+  }
+
+  /**
+   * Test HttpSolrServer's delegation token support
+   */
+  @Test
+  public void testDelegationTokenSolrClient() throws Exception {
+    // Get token
+    String token = getDelegationToken(null, "bar", solrClientPrimary);
+    assertNotNull(token);
+
+    SolrRequest request = getAdminRequest(new ModifiableSolrParams());
+
+    // test without token
+    HttpSolrClient ss =
+        new HttpSolrClient.Builder(solrClientPrimary.getBaseURL().toString())
+            .withResponseParser(solrClientPrimary.getParser())
+            .build();
+    try {
+      doSolrRequest(ss, request, ErrorCode.UNAUTHORIZED.code);
+    } finally {
+      ss.close();
+    }
+
+    ss = new HttpSolrClient.Builder(solrClientPrimary.getBaseURL().toString())
+        .withDelegationToken(token)
+        .withResponseParser(solrClientPrimary.getParser())
+        .build();
+    try {
+      // test with token via property
+      doSolrRequest(ss, request, HttpStatus.SC_OK);
+
+      // test with param -- should throw an exception
+      ModifiableSolrParams tokenParam = new ModifiableSolrParams();
+      tokenParam.set("delegation", "invalidToken");
+      try {
+        doSolrRequest(ss, getAdminRequest(tokenParam), ErrorCode.FORBIDDEN.code);
+        Assert.fail("Expected exception");
+      } catch (IllegalArgumentException ex) {}
+    } finally {
+      ss.close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/core/src/test/org/apache/solr/cloud/TestSolrCloudWithKerberosAlt.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestSolrCloudWithKerberosAlt.java b/solr/core/src/test/org/apache/solr/cloud/TestSolrCloudWithKerberosAlt.java
index 6ac2254..c505b51 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestSolrCloudWithKerberosAlt.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestSolrCloudWithKerberosAlt.java
@@ -17,15 +17,12 @@
 package org.apache.solr.cloud;
 
 import java.io.File;
+import java.lang.invoke.MethodHandles;
 import java.nio.charset.StandardCharsets;
 import java.util.List;
-import java.util.Locale;
 import java.util.Properties;
 
-import javax.security.auth.login.Configuration;
-
 import org.apache.commons.io.FileUtils;
-import org.apache.hadoop.minikdc.MiniKdc;
 import org.apache.lucene.index.TieredMergePolicy;
 import org.apache.lucene.util.Constants;
 import org.apache.lucene.util.LuceneTestCase;
@@ -49,6 +46,8 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.RuleChain;
 import org.junit.rules.TestRule;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters;
 import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
@@ -67,7 +66,7 @@ import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
 @LuceneTestCase.SuppressSysoutChecks(bugUrl = "Solr logs to JUL")
 public class TestSolrCloudWithKerberosAlt extends LuceneTestCase {
 
-  private final Configuration originalConfig = Configuration.getConfiguration();
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   protected final int NUM_SERVERS;
   protected final int NUM_SHARDS;
   protected final int REPLICATION_FACTOR;
@@ -78,10 +77,8 @@ public class TestSolrCloudWithKerberosAlt extends LuceneTestCase {
     REPLICATION_FACTOR = 1;
   }
 
-  private MiniKdc kdc;
+  private KerberosTestServices kerberosTestServices;
 
-  private Locale savedLocale; // in case locale is broken and we need to fill in a working locale
-  
   @Rule
   public TestRule solrTestRules = RuleChain
       .outerRule(new SystemPropertiesRestoreRule());
@@ -98,7 +95,6 @@ public class TestSolrCloudWithKerberosAlt extends LuceneTestCase {
 
   @Override
   public void setUp() throws Exception {
-    savedLocale = KerberosTestUtil.overrideLocaleIfNotSpportedByMiniKdc();
     super.setUp();
     setupMiniKdc();
   }
@@ -106,12 +102,15 @@ public class TestSolrCloudWithKerberosAlt extends LuceneTestCase {
   private void setupMiniKdc() throws Exception {
     System.setProperty("solr.jaas.debug", "true");
     String kdcDir = createTempDir()+File.separator+"minikdc";
-    kdc = KerberosTestUtil.getKdc(new File(kdcDir));
+    String solrClientPrincipal = "solr";
     File keytabFile = new File(kdcDir, "keytabs");
+    kerberosTestServices = KerberosTestServices.builder()
+        .withKdc(new File(kdcDir))
+        .withJaasConfiguration(solrClientPrincipal, keytabFile, "SolrClient")
+        .build();
     String solrServerPrincipal = "HTTP/127.0.0.1";
-    String solrClientPrincipal = "solr";
-    kdc.start();
-    kdc.createPrincipal(keytabFile, solrServerPrincipal, solrClientPrincipal);
+    kerberosTestServices.start();
+    kerberosTestServices.getKdc().createPrincipal(keytabFile, solrServerPrincipal, solrClientPrincipal);
 
     String jaas = "SolrClient {\n"
         + " com.sun.security.auth.module.Krb5LoginModule required\n"
@@ -124,9 +123,6 @@ public class TestSolrCloudWithKerberosAlt extends LuceneTestCase {
         + " principal=\"" + solrClientPrincipal + "\";\n"
         + "};";
 
-    Configuration conf = new KerberosTestUtil.JaasConfiguration(solrClientPrincipal, keytabFile, "SolrClient");
-    Configuration.setConfiguration(conf);
-
     String jaasFilePath = kdcDir+File.separator+"jaas-client.conf";
     FileUtils.write(new File(jaasFilePath), jaas, StandardCharsets.UTF_8);
     System.setProperty("java.security.auth.login.config", jaasFilePath);
@@ -135,6 +131,9 @@ public class TestSolrCloudWithKerberosAlt extends LuceneTestCase {
     System.setProperty("solr.kerberos.principal", solrServerPrincipal);
     System.setProperty("solr.kerberos.keytab", keytabFile.getAbsolutePath());
     System.setProperty("authenticationPlugin", "org.apache.solr.security.KerberosPlugin");
+    boolean enableDt = random().nextBoolean();
+    log.info("Enable delegation token: " + enableDt);
+    System.setProperty("solr.kerberos.delegation.token.enabled", new Boolean(enableDt).toString());
     // Extracts 127.0.0.1 from HTTP/127.0.0.1@EXAMPLE.COM
     System.setProperty("solr.kerberos.name.rules", "RULE:[1:$1@$0](.*EXAMPLE.COM)s/@.*//"
         + "\nRULE:[2:$2@$0](.*EXAMPLE.COM)s/@.*//"
@@ -240,11 +239,7 @@ public class TestSolrCloudWithKerberosAlt extends LuceneTestCase {
     System.clearProperty("authenticationPlugin");
     System.clearProperty("solr.kerberos.name.rules");
     System.clearProperty("solr.jaas.debug");
-    Configuration.setConfiguration(this.originalConfig);
-    if (kdc != null) {
-      kdc.stop();
-    }
-    Locale.setDefault(savedLocale);
+    kerberosTestServices.stop();
     super.tearDown();
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/core/src/test/org/apache/solr/cloud/VMParamsZkACLAndCredentialsProvidersTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/VMParamsZkACLAndCredentialsProvidersTest.java b/solr/core/src/test/org/apache/solr/cloud/VMParamsZkACLAndCredentialsProvidersTest.java
index 31919a8..95422fa 100644
--- a/solr/core/src/test/org/apache/solr/cloud/VMParamsZkACLAndCredentialsProvidersTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/VMParamsZkACLAndCredentialsProvidersTest.java
@@ -21,6 +21,7 @@ import java.lang.invoke.MethodHandles;
 import java.nio.charset.Charset;
 
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.cloud.SecurityAwareZkACLProvider;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.VMParamsAllAndReadonlyDigestZkACLProvider;
 import org.apache.solr.common.cloud.VMParamsSingleSetCredentialsDigestZkCredentialsProvider;
@@ -76,6 +77,8 @@ public class VMParamsZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
     zkClient = new SolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT);
     zkClient.create("/protectedCreateNode", "content".getBytes(DATA_ENCODING), CreateMode.PERSISTENT, false);
     zkClient.makePath("/protectedMakePathNode", "content".getBytes(DATA_ENCODING), CreateMode.PERSISTENT, false);
+
+    zkClient.create(SecurityAwareZkACLProvider.SECURITY_ZNODE_PATH, "content".getBytes(DATA_ENCODING), CreateMode.PERSISTENT, false);
     zkClient.close();
     
     clearSecuritySystemProperties();
@@ -106,7 +109,9 @@ public class VMParamsZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
     
     SolrZkClient zkClient = new SolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT);
     try {
-      doTest(zkClient, false, false, false, false, false);
+      doTest(zkClient,
+          false, false, false, false, false,
+          false, false, false, false, false);
     } finally {
       zkClient.close();
     }
@@ -118,7 +123,9 @@ public class VMParamsZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
     
     SolrZkClient zkClient = new SolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT);
     try {
-      doTest(zkClient, false, false, false, false, false);
+      doTest(zkClient,
+          false, false, false, false, false,
+          false, false, false, false, false);
     } finally {
       zkClient.close();
     }
@@ -130,7 +137,9 @@ public class VMParamsZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
 
     SolrZkClient zkClient = new SolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT);
     try {
-      doTest(zkClient, true, true, true, true, true);
+      doTest(zkClient,
+          true, true, true, true, true,
+          true, true, true, true, true);
     } finally {
       zkClient.close();
     }
@@ -142,17 +151,23 @@ public class VMParamsZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
 
     SolrZkClient zkClient = new SolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT);
     try {
-      doTest(zkClient, true, true, false, false, false);
+      doTest(zkClient,
+          true, true, false, false, false,
+          false, false, false, false, false);
     } finally {
       zkClient.close();
     }
   }
     
-  protected static void doTest(SolrZkClient zkClient, boolean getData, boolean list, boolean create, boolean setData, boolean delete) throws Exception {
+  protected static void doTest(
+      SolrZkClient zkClient,
+      boolean getData, boolean list, boolean create, boolean setData, boolean delete,
+      boolean secureGet, boolean secureList, boolean secureCreate, boolean secureSet, boolean secureDelete) throws Exception {
     doTest(zkClient, "/protectedCreateNode", getData, list, create, setData, delete);
     doTest(zkClient, "/protectedMakePathNode", getData, list, create, setData, delete);
     doTest(zkClient, "/unprotectedCreateNode", true, true, true, true, delete);
     doTest(zkClient, "/unprotectedMakePathNode", true, true, true, true, delete);
+    doTest(zkClient, SecurityAwareZkACLProvider.SECURITY_ZNODE_PATH, secureGet, secureList, secureCreate, secureSet, secureDelete);
   }
   
   protected static void doTest(SolrZkClient zkClient, String path, boolean getData, boolean list, boolean create, boolean setData, boolean delete) throws Exception {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/core/src/test/org/apache/solr/security/HttpParamDelegationTokenAuthenticationHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/HttpParamDelegationTokenAuthenticationHandler.java b/solr/core/src/test/org/apache/solr/security/HttpParamDelegationTokenAuthenticationHandler.java
new file mode 100644
index 0000000..7c5c94a
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/security/HttpParamDelegationTokenAuthenticationHandler.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.security;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.hadoop.security.authentication.server.AuthenticationHandler;
+import org.apache.hadoop.security.authentication.server.AuthenticationToken;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticationHandler;
+
+import org.apache.http.NameValuePair;
+import org.apache.http.client.utils.URLEncodedUtils;
+
+/**
+ * AuthenticationHandler that supports delegation tokens and simple
+ * authentication via the "user" http parameter
+ */
+public class HttpParamDelegationTokenAuthenticationHandler extends
+    DelegationTokenAuthenticationHandler {
+
+  public static final String USER_PARAM = "user";
+
+  public HttpParamDelegationTokenAuthenticationHandler() {
+    super(new HttpParamAuthenticationHandler());
+  }
+
+  @Override
+  public void init(Properties config) throws ServletException {
+    Properties conf = new Properties();
+    for (Map.Entry entry : config.entrySet()) {
+      conf.setProperty((String) entry.getKey(), (String) entry.getValue());
+    }
+    conf.setProperty(TOKEN_KIND, KerberosPlugin.DELEGATION_TOKEN_TYPE_DEFAULT);
+    super.init(conf);
+  }
+ 
+  private static String getHttpParam(HttpServletRequest request, String param) {
+    List<NameValuePair> pairs =
+      URLEncodedUtils.parse(request.getQueryString(), Charset.forName("UTF-8"));
+    for (NameValuePair nvp : pairs) {
+      if(param.equals(nvp.getName())) {
+        return nvp.getValue();
+      }
+    }
+    return null;
+  }
+
+  private static class HttpParamAuthenticationHandler
+      implements AuthenticationHandler {
+
+    @Override
+    public String getType() {
+      return "dummy";
+    }
+
+    @Override
+    public void init(Properties config) throws ServletException {
+    }
+
+    @Override
+    public void destroy() {
+    }
+
+    @Override
+    public boolean managementOperation(AuthenticationToken token,
+        HttpServletRequest request, HttpServletResponse response)
+        throws IOException, AuthenticationException {
+      return false;
+    }
+
+    @Override
+    public AuthenticationToken authenticate(HttpServletRequest request,
+        HttpServletResponse response)
+        throws IOException, AuthenticationException {
+      AuthenticationToken token = null;
+      String userName = getHttpParam(request, USER_PARAM);
+      if (userName != null) {
+        return new AuthenticationToken(userName, userName, "test");
+      } else {
+        response.setStatus(HttpServletResponse.SC_UNAUTHORIZED);
+        response.setHeader("WWW-Authenticate", "dummy");
+      }
+      return token;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/core/src/test/org/apache/solr/security/MockAuthenticationPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/MockAuthenticationPlugin.java b/solr/core/src/test/org/apache/solr/security/MockAuthenticationPlugin.java
index e3cf7bd..3013086 100644
--- a/solr/core/src/test/org/apache/solr/security/MockAuthenticationPlugin.java
+++ b/solr/core/src/test/org/apache/solr/security/MockAuthenticationPlugin.java
@@ -20,11 +20,16 @@ import javax.servlet.FilterChain;
 import javax.servlet.ServletException;
 import javax.servlet.ServletRequest;
 import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletRequestWrapper;
 import java.io.IOException;
 import java.security.Principal;
 import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.function.Predicate;
 
+import org.apache.http.auth.BasicUserPrincipal;
+
 public class MockAuthenticationPlugin extends AuthenticationPlugin {
   static Predicate<ServletRequest> predicate;
 
@@ -33,7 +38,7 @@ public class MockAuthenticationPlugin extends AuthenticationPlugin {
   }
 
   @Override
-  public void doAuthenticate(ServletRequest request, ServletResponse response, FilterChain filterChain) throws IOException, ServletException {
+  public boolean doAuthenticate(ServletRequest request, ServletResponse response, FilterChain filterChain) throws IOException, ServletException {
     String user = null;
     if (predicate != null) {
       if (predicate.test(request)) {
@@ -41,9 +46,32 @@ public class MockAuthenticationPlugin extends AuthenticationPlugin {
         request.removeAttribute(Principal.class.getName());
       }
     }
-    forward(user, request, response, filterChain);
+
+    final FilterChain ffc = filterChain;
+    final AtomicBoolean requestContinues = new AtomicBoolean(false);
+    forward(user, request, response, new FilterChain() {
+      @Override
+      public void doFilter(ServletRequest req, ServletResponse res) throws IOException, ServletException {
+        ffc.doFilter(req, res);
+        requestContinues.set(true);
+      }
+    });
+    return requestContinues.get();
   }
 
+  protected void forward(String user, ServletRequest  req, ServletResponse rsp,
+                                    FilterChain chain) throws IOException, ServletException {
+    if(user != null) {
+      final Principal p = new BasicUserPrincipal(user);
+      req = new HttpServletRequestWrapper((HttpServletRequest) req) {
+        @Override
+        public Principal getUserPrincipal() {
+          return p;
+        }
+      };
+    }
+    chain.doFilter(req,rsp);
+  }
 
   @Override
   public void close() throws IOException {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/licenses/curator-recipes-2.8.0.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/curator-recipes-2.8.0.jar.sha1 b/solr/licenses/curator-recipes-2.8.0.jar.sha1
new file mode 100644
index 0000000..82d8946
--- /dev/null
+++ b/solr/licenses/curator-recipes-2.8.0.jar.sha1
@@ -0,0 +1 @@
+c563e25fb37f85a6b029bc9746e75573640474fb

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/licenses/curator-recipes-NOTICE.txt
----------------------------------------------------------------------
diff --git a/solr/licenses/curator-recipes-NOTICE.txt b/solr/licenses/curator-recipes-NOTICE.txt
new file mode 100644
index 0000000..f568d0f
--- /dev/null
+++ b/solr/licenses/curator-recipes-NOTICE.txt
@@ -0,0 +1,5 @@
+Apache Curator
+Copyright 2013-2014 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/solrj/ivy.xml
----------------------------------------------------------------------
diff --git a/solr/solrj/ivy.xml b/solr/solrj/ivy.xml
index ceefcc3..f2296c5 100644
--- a/solr/solrj/ivy.xml
+++ b/solr/solrj/ivy.xml
@@ -1,3 +1,4 @@
+<?xml version="1.0" encoding="utf-8"?>
 <!--
    Licensed to the Apache Software Foundation (ASF) under one
    or more contributor license agreements.  See the NOTICE file
@@ -40,6 +41,10 @@
 
     <dependency org="org.slf4j" name="slf4j-log4j12" rev="${/org.slf4j/slf4j-log4j12}" conf="test"/>
 
+    <dependency org="com.fasterxml.jackson.core" name="jackson-annotations"  rev="${/com.fasterxml.jackson.core/jackson-annotations}"   conf="compile"/>
+    <dependency org="com.fasterxml.jackson.core" name="jackson-core" rev="${/com.fasterxml.jackson.core/jackson-core}" conf="compile"/>
+    <dependency org="com.fasterxml.jackson.core" name="jackson-databind" rev="${/com.fasterxml.jackson.core/jackson-databind}" conf="compile"/>
+    <dependency org="com.google.guava" name="guava" rev="${/com.google.guava/guava}" conf="compile"/>
     <exclude org="*" ext="*" matcher="regexp" type="${ivy.exclude.types}"/>
   </dependencies>
 </ivy-module>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java
index 9b0cf8d..222119c 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java
@@ -23,6 +23,7 @@ import java.lang.invoke.MethodHandles;
 import java.net.ConnectException;
 import java.net.SocketTimeoutException;
 import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Iterator;
@@ -30,6 +31,7 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Locale;
 import java.util.Set;
+import java.util.TreeSet;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
 
@@ -743,7 +745,44 @@ public class HttpSolrClient extends SolrClient {
       super(code, "Error from server at " + remoteHost + ": " + msg, th);
     }
   }
-  
+
+  private static class DelegationTokenHttpSolrClient extends HttpSolrClient {
+    private final String DELEGATION_TOKEN_PARAM = "delegation";
+    private final String delegationToken;
+
+    public DelegationTokenHttpSolrClient(String baseURL,
+                                         HttpClient client,
+                                         ResponseParser parser,
+                                         boolean allowCompression,
+                                         String delegationToken) {
+      super(baseURL, client, parser, allowCompression);
+      if (delegationToken == null) {
+        throw new IllegalArgumentException("Delegation token cannot be null");
+      }
+      this.delegationToken = delegationToken;
+      setQueryParams(new TreeSet<String>(Arrays.asList(DELEGATION_TOKEN_PARAM)));
+      invariantParams = new ModifiableSolrParams();
+      invariantParams.set(DELEGATION_TOKEN_PARAM, delegationToken);
+    }
+
+    @Override
+    protected HttpRequestBase createMethod(final SolrRequest request, String collection) throws IOException, SolrServerException {
+      SolrParams params = request.getParams();
+      if (params.getParams(DELEGATION_TOKEN_PARAM) != null) {
+        throw new IllegalArgumentException(DELEGATION_TOKEN_PARAM + " parameter not supported");
+      }
+      return super.createMethod(request, collection);
+    }
+
+    @Override
+    public void setQueryParams(Set<String> queryParams) {
+      if (queryParams == null || !queryParams.contains(DELEGATION_TOKEN_PARAM)) {
+        throw new IllegalArgumentException("Query params must contain " + DELEGATION_TOKEN_PARAM);
+      }
+      super.setQueryParams(queryParams);
+    }
+  }
+
   /**
    * Constructs {@link HttpSolrClient} instances from provided configuration.
    */
@@ -752,6 +791,7 @@ public class HttpSolrClient extends SolrClient {
     private HttpClient httpClient;
     private ResponseParser responseParser;
     private boolean compression;
+    private String delegationToken;
     
     /**
      * Create a Builder object, based on the provided Solr URL.
@@ -788,7 +828,14 @@ public class HttpSolrClient extends SolrClient {
       this.compression = compression;
       return this;
     }
-    
+
+    /**
+     * Use a delegation token for authenticating via the KerberosPlugin
+     */
+    public Builder withDelegationToken(String delegationToken) {
+      this.delegationToken = delegationToken;
+      return this;
+    }
     /**
      * Create a {@link HttpSolrClient} based on provided configuration.
      */
@@ -796,7 +843,11 @@ public class HttpSolrClient extends SolrClient {
       if (baseSolrUrl == null) {
         throw new IllegalArgumentException("Cannot create HttpSolrClient without a valid baseSolrUrl!");
       }
-      return new HttpSolrClient(baseSolrUrl, httpClient, responseParser, compression);
+      if (delegationToken == null) {
+        return new HttpSolrClient(baseSolrUrl, httpClient, responseParser, compression);
+      } else {
+        return new DelegationTokenHttpSolrClient(baseSolrUrl, httpClient, responseParser, compression, delegationToken);
+      }
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Krb5HttpClientBuilder.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Krb5HttpClientBuilder.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Krb5HttpClientBuilder.java
index 9d5a926..84fe5f9 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Krb5HttpClientBuilder.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Krb5HttpClientBuilder.java
@@ -26,6 +26,7 @@ import java.util.Set;
 import javax.security.auth.login.AppConfigurationEntry;
 import javax.security.auth.login.Configuration;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.http.HttpEntity;
 import org.apache.http.HttpEntityEnclosingRequest;
 import org.apache.http.HttpRequestInterceptor;
@@ -51,12 +52,21 @@ public class Krb5HttpClientBuilder  {
   public static final String LOGIN_CONFIG_PROP = "java.security.auth.login.config";
   private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   
-  private static final Configuration jaasConfig = new SolrJaasConfiguration();
+  private static Configuration jaasConfig = new SolrJaasConfiguration();
 
   public Krb5HttpClientBuilder() {
 
   }
-  
+
+  /**
+   * The jaasConfig is static, which makes it problematic for testing in the same jvm.
+   * Call this function to regenerate the static config (this is not thread safe).
+   */
+  @VisibleForTesting
+  public static void regenerateJaasConfiguration() {
+    jaasConfig = new SolrJaasConfiguration();
+  }
+
   public SolrHttpClientBuilder getBuilder() {
     return getBuilder(HttpClientUtil.getHttpClientBuilder());
   }
@@ -104,9 +114,9 @@ public class Krb5HttpClientBuilder  {
             return null;
           }
         };
-        
+
         HttpClientUtil.setCookiePolicy(SolrPortAwareCookieSpecFactory.POLICY_NAME);
-        
+
         builder.setCookieSpecRegistryProvider(() -> {
           SolrPortAwareCookieSpecFactory cookieFactory = new SolrPortAwareCookieSpecFactory();
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/solrj/src/java/org/apache/solr/client/solrj/request/DelegationTokenRequest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/DelegationTokenRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/DelegationTokenRequest.java
new file mode 100644
index 0000000..6d2f0cb
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/DelegationTokenRequest.java
@@ -0,0 +1,152 @@
+/*
+ * 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.request;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.impl.NoOpResponseParser;
+import org.apache.solr.client.solrj.response.DelegationTokenResponse;
+
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.ContentStream;
+
+/**
+ * Class for making Solr delegation token requests.
+ *
+ * @since Solr 6.2
+ */
+public abstract class DelegationTokenRequest
+    <Q extends DelegationTokenRequest<Q,R>, R extends DelegationTokenResponse>
+    extends SolrRequest<R> {
+
+  protected static final String OP_KEY = "op";
+  protected static final String TOKEN_KEY = "token";
+
+  public DelegationTokenRequest(METHOD m) {
+    // path doesn't really matter -- the filter will respond to any path.
+    // setting the path to admin/collections lets us pass through CloudSolrServer
+    // without having to specify a collection (that may not even exist yet).
+    super(m, "/admin/collections");
+  }
+
+  protected abstract Q getThis();
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public Collection<ContentStream> getContentStreams() throws IOException {
+    return null;
+  }
+
+  @Override
+  protected abstract R createResponse(SolrClient client);
+
+  public static class Get extends DelegationTokenRequest<Get, DelegationTokenResponse.Get> {
+    protected String renewer;
+
+    public Get() {
+      this(null);
+    }
+
+    public Get(String renewer) {
+      super(METHOD.GET);
+      this.renewer = renewer;
+      setResponseParser(new DelegationTokenResponse.JsonMapResponseParser());
+      setQueryParams(new TreeSet<String>(Arrays.asList(OP_KEY)));
+    }
+
+    @Override
+    protected Get getThis() {
+      return this;
+    }
+
+    @Override
+    public SolrParams getParams() {
+      ModifiableSolrParams params = new ModifiableSolrParams();
+      params.set(OP_KEY, "GETDELEGATIONTOKEN");
+      if (renewer != null) params.set("renewer", renewer);
+      return params;
+    }
+
+    @Override
+    public DelegationTokenResponse.Get createResponse(SolrClient client) { return new DelegationTokenResponse.Get(); }
+  }
+
+  public static class Renew extends DelegationTokenRequest<Renew, DelegationTokenResponse.Renew> {
+    protected String token;
+
+    @Override
+    protected Renew getThis() {
+      return this;
+    }
+
+    public Renew(String token) {
+      super(METHOD.PUT);
+      this.token = token;
+      setResponseParser(new DelegationTokenResponse.JsonMapResponseParser());
+      setQueryParams(new TreeSet<String>(Arrays.asList(OP_KEY, TOKEN_KEY)));
+    }
+
+    @Override
+    public SolrParams getParams() {
+      ModifiableSolrParams params = new ModifiableSolrParams();
+      params.set(OP_KEY, "RENEWDELEGATIONTOKEN");
+      params.set(TOKEN_KEY, token);
+      return params;
+    }
+
+    @Override
+    public DelegationTokenResponse.Renew createResponse(SolrClient client) { return new DelegationTokenResponse.Renew(); }
+  }
+
+  public static class Cancel extends DelegationTokenRequest<Cancel, DelegationTokenResponse.Cancel> {
+    protected String token;
+
+    public Cancel(String token) {
+      super(METHOD.PUT);
+      this.token = token;
+      setResponseParser(new NoOpResponseParser());
+      Set<String> queryParams = new TreeSet<String>();
+      setQueryParams(new TreeSet<String>(Arrays.asList(OP_KEY, TOKEN_KEY)));
+    }
+
+    @Override
+    protected Cancel getThis() {
+      return this;
+    }
+
+    @Override
+    public SolrParams getParams() {
+      ModifiableSolrParams params = new ModifiableSolrParams();
+      params.set(OP_KEY, "CANCELDELEGATIONTOKEN");
+      params.set(TOKEN_KEY, token);
+      return params;
+    }
+
+    @Override
+    public DelegationTokenResponse.Cancel createResponse(SolrClient client) { return new DelegationTokenResponse.Cancel(); }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/solrj/src/java/org/apache/solr/client/solrj/response/DelegationTokenResponse.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/response/DelegationTokenResponse.java b/solr/solrj/src/java/org/apache/solr/client/solrj/response/DelegationTokenResponse.java
new file mode 100644
index 0000000..c80e2eb
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/response/DelegationTokenResponse.java
@@ -0,0 +1,108 @@
+/*
+ * 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.response;
+
+import org.apache.solr.client.solrj.ResponseParser;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.util.NamedList;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.Reader;
+import java.util.Map;
+
+/**
+ * Delegation Token responses
+ */
+public abstract class DelegationTokenResponse extends SolrResponseBase {
+
+  public static class Get extends DelegationTokenResponse {
+
+    /**
+     * Get the urlString to be used as the delegation token
+     */
+    public String getDelegationToken() {
+      try {
+        Map map = (Map)getResponse().get("Token");
+        if (map != null) {
+          return (String)map.get("urlString");
+        }
+      } catch (ClassCastException e) {
+        throw new SolrException (SolrException.ErrorCode.SERVER_ERROR,
+          "parsing error", e);
+      }
+      return null;
+    }
+  }
+
+  public static class Renew extends DelegationTokenResponse {
+    public Long getExpirationTime() {
+      try {
+        return (Long)getResponse().get("long");
+      } catch (ClassCastException e) {
+        throw new SolrException (SolrException.ErrorCode.SERVER_ERROR,
+          "parsing error", e);
+      }
+    }
+  }
+
+  public static class Cancel extends DelegationTokenResponse {
+  }
+
+  /**
+   * ResponseParser for JsonMaps.  Used for Get and Renew DelegationToken responses.
+   */
+  public static class JsonMapResponseParser extends ResponseParser {
+    @Override
+    public String getWriterType() {
+      return "json";
+    }
+
+    @Override
+    public NamedList<Object> processResponse(InputStream body, String encoding) {
+      ObjectMapper mapper = new ObjectMapper();
+      Map map = null;
+      try {
+        map = mapper.readValue(body, Map.class);
+      } catch (IOException e) {
+        throw new SolrException (SolrException.ErrorCode.SERVER_ERROR,
+          "parsing error", e);
+      }
+      NamedList<Object> list = new NamedList<Object>();
+      list.addAll(map);
+      return list;
+    }
+
+    @Override
+    public NamedList<Object> processResponse(Reader reader) {
+      throw new RuntimeException("Cannot handle character stream");
+    }
+
+    @Override
+    public String getContentType() {
+      return "application/json";
+    }
+
+    @Override
+    public String getVersion() {
+      return "1";
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/solrj/src/java/org/apache/solr/common/cloud/SaslZkACLProvider.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/SaslZkACLProvider.java b/solr/solrj/src/java/org/apache/solr/common/cloud/SaslZkACLProvider.java
index eaccb8c..c67ad12 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/SaslZkACLProvider.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/SaslZkACLProvider.java
@@ -30,19 +30,22 @@ import org.apache.zookeeper.data.Id;
  * configurations have already been set up and will not be modified, or
  * where configuration changes are controlled via Solr APIs.
  */
-public class SaslZkACLProvider extends DefaultZkACLProvider {
+public class SaslZkACLProvider extends SecurityAwareZkACLProvider {
 
   private static String superUser = System.getProperty("solr.authorization.superuser", "solr");
 
   @Override
-  protected List<ACL> createGlobalACLsToAdd() {
-    List<ACL> result = new ArrayList<ACL>();
-    result.add(new ACL(ZooDefs.Perms.ALL, new Id("sasl", superUser)));
-    result.add(new ACL(ZooDefs.Perms.READ, ZooDefs.Ids.ANYONE_ID_UNSAFE));
+  protected List<ACL> createNonSecurityACLsToAdd() {
+    List<ACL> ret = new ArrayList<ACL>();
+    ret.add(new ACL(ZooDefs.Perms.ALL, new Id("sasl", superUser)));
+    ret.add(new ACL(ZooDefs.Perms.READ, ZooDefs.Ids.ANYONE_ID_UNSAFE));
+    return ret;
+  }
 
-    if (result.isEmpty()) {
-      result = super.createGlobalACLsToAdd();
-    }
-    return result;
+  @Override
+  protected List<ACL> createSecurityACLsToAdd() {
+    List<ACL> ret = new ArrayList<ACL>();
+    ret.add(new ACL(ZooDefs.Perms.ALL, new Id("sasl", superUser)));
+    return ret;
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/solrj/src/java/org/apache/solr/common/cloud/SecurityAwareZkACLProvider.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/SecurityAwareZkACLProvider.java b/solr/solrj/src/java/org/apache/solr/common/cloud/SecurityAwareZkACLProvider.java
new file mode 100644
index 0000000..1c74d94
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/SecurityAwareZkACLProvider.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.solr.common.cloud;
+
+import java.util.List;
+
+import org.apache.zookeeper.data.ACL;
+
+/**
+ * {@link ZkACLProvider} capable of returning a different set of
+ * {@link ACL}s for security-related znodes (default: subtree under /security)
+ * vs non-security-related znodes.
+ */
+public abstract class SecurityAwareZkACLProvider implements ZkACLProvider {
+  public static final String SECURITY_ZNODE_PATH = "/security";
+
+  private List<ACL> nonSecurityACLsToAdd;
+  private List<ACL> securityACLsToAdd;
+
+
+  @Override
+  public List<ACL> getACLsToAdd(String zNodePath) {
+    if (isSecurityZNodePath(zNodePath)) {
+      return getSecurityACLsToAdd();
+    } else {
+      return getNonSecurityACLsToAdd();
+    }
+  }
+
+  protected boolean isSecurityZNodePath(String zNodePath) {
+    if (zNodePath != null
+        && (zNodePath.equals(SECURITY_ZNODE_PATH) || zNodePath.startsWith(SECURITY_ZNODE_PATH + "/"))) {
+      return true;
+    }
+    return false;
+  }
+
+  /**
+   * @return Set of ACLs to return for non-security related znodes
+   */
+  protected abstract List<ACL> createNonSecurityACLsToAdd();
+
+  /**
+   * @return Set of ACLs to return security-related znodes
+   */
+  protected abstract List<ACL> createSecurityACLsToAdd();
+
+  private List<ACL> getNonSecurityACLsToAdd() {
+    if (nonSecurityACLsToAdd == null) {
+      synchronized (this) {
+        if (nonSecurityACLsToAdd == null) nonSecurityACLsToAdd = createNonSecurityACLsToAdd();
+      }
+    }
+    return nonSecurityACLsToAdd;
+  }
+
+  private List<ACL> getSecurityACLsToAdd() {
+    if (securityACLsToAdd == null) {
+      synchronized (this) {
+        if (securityACLsToAdd == null) securityACLsToAdd = createSecurityACLsToAdd();
+      }
+    }
+    return securityACLsToAdd;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java b/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
index af6f9be..516b7b9 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
@@ -122,12 +122,12 @@ public class SolrZkClient implements Closeable {
 
   public SolrZkClient(String zkServerAddress, int zkClientTimeout, int clientConnectTimeout,
       ZkClientConnectionStrategy strat, final OnReconnect onReconnect, BeforeReconnect beforeReconnect, ZkACLProvider zkACLProvider) {
-    this.zkClientConnectionStrategy = strat;
     this.zkServerAddress = zkServerAddress;
     
     if (strat == null) {
       strat = new DefaultConnectionStrategy();
     }
+    this.zkClientConnectionStrategy = strat;
 
     if (!strat.hasZkCredentialsToAddAutomatically()) {
       ZkCredentialsProvider zkCredentialsToAddAutomatically = createZkCredentialsToAddAutomatically();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/solrj/src/java/org/apache/solr/common/cloud/VMParamsAllAndReadonlyDigestZkACLProvider.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/VMParamsAllAndReadonlyDigestZkACLProvider.java b/solr/solrj/src/java/org/apache/solr/common/cloud/VMParamsAllAndReadonlyDigestZkACLProvider.java
index f6f491b..8866245 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/VMParamsAllAndReadonlyDigestZkACLProvider.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/VMParamsAllAndReadonlyDigestZkACLProvider.java
@@ -20,13 +20,14 @@ import java.security.NoSuchAlgorithmException;
 import java.util.ArrayList;
 import java.util.List;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.solr.common.StringUtils;
 import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Id;
 import org.apache.zookeeper.server.auth.DigestAuthenticationProvider;
 
-public class VMParamsAllAndReadonlyDigestZkACLProvider extends DefaultZkACLProvider {
+public class VMParamsAllAndReadonlyDigestZkACLProvider extends SecurityAwareZkACLProvider {
 
   public static final String DEFAULT_DIGEST_READONLY_USERNAME_VM_PARAM_NAME = "zkDigestReadonlyUsername";
   public static final String DEFAULT_DIGEST_READONLY_PASSWORD_VM_PARAM_NAME = "zkDigestReadonlyPassword";
@@ -53,29 +54,56 @@ public class VMParamsAllAndReadonlyDigestZkACLProvider extends DefaultZkACLProvi
     this.zkDigestReadonlyPasswordVMParamName = zkDigestReadonlyPasswordVMParamName;
   }
 
+  /**
+   * @return Set of ACLs to return for non-security related znodes
+   */
+  @Override
+  protected List<ACL> createNonSecurityACLsToAdd() {
+    return createACLsToAdd(true);
+  }
 
+  /**
+   * @return Set of ACLs to return security-related znodes
+   */
   @Override
-  protected List<ACL> createGlobalACLsToAdd() {
-    try {
+  protected List<ACL> createSecurityACLsToAdd() {
+    return createACLsToAdd(false);
+  }
+
+  protected List<ACL> createACLsToAdd(boolean includeReadOnly) {
+    String digestAllUsername = System.getProperty(zkDigestAllUsernameVMParamName);
+    String digestAllPassword = System.getProperty(zkDigestAllPasswordVMParamName);
+    String digestReadonlyUsername = System.getProperty(zkDigestReadonlyUsernameVMParamName);
+    String digestReadonlyPassword = System.getProperty(zkDigestReadonlyPasswordVMParamName);
+
+    return createACLsToAdd(includeReadOnly,
+        digestAllUsername, digestAllPassword,
+        digestReadonlyUsername, digestReadonlyPassword);
+  }
+
+  @VisibleForTesting
+  protected List<ACL> createACLsToAdd(boolean includeReadOnly,
+                                      String digestAllUsername, String digestAllPassword,
+                                      String digestReadonlyUsername, String digestReadonlyPassword) {
+
+      try {
       List<ACL> result = new ArrayList<ACL>();
   
       // Not to have to provide too much credentials and ACL information to the process it is assumed that you want "ALL"-acls
       // added to the user you are using to connect to ZK (if you are using VMParamsSingleSetCredentialsDigestZkCredentialsProvider)
-      String digestAllUsername = System.getProperty(zkDigestAllUsernameVMParamName);
-      String digestAllPassword = System.getProperty(zkDigestAllPasswordVMParamName);
       if (!StringUtils.isEmpty(digestAllUsername) && !StringUtils.isEmpty(digestAllPassword)) {
         result.add(new ACL(ZooDefs.Perms.ALL, new Id("digest", DigestAuthenticationProvider.generateDigest(digestAllUsername + ":" + digestAllPassword))));
       }
-  
-      // Besides that support for adding additional "READONLY"-acls for another user
-      String digestReadonlyUsername = System.getProperty(zkDigestReadonlyUsernameVMParamName);
-      String digestReadonlyPassword = System.getProperty(zkDigestReadonlyPasswordVMParamName);
-      if (!StringUtils.isEmpty(digestReadonlyUsername) && !StringUtils.isEmpty(digestReadonlyPassword)) {
-        result.add(new ACL(ZooDefs.Perms.READ, new Id("digest", DigestAuthenticationProvider.generateDigest(digestReadonlyUsername + ":" + digestReadonlyPassword))));
+
+      if (includeReadOnly) {
+        // Besides that support for adding additional "READONLY"-acls for another user
+        if (!StringUtils.isEmpty(digestReadonlyUsername) && !StringUtils.isEmpty(digestReadonlyPassword)) {
+          result.add(new ACL(ZooDefs.Perms.READ, new Id("digest", DigestAuthenticationProvider.generateDigest(digestReadonlyUsername + ":" + digestReadonlyPassword))));
+        }
       }
       
       if (result.isEmpty()) {
-        result = super.createGlobalACLsToAdd();
+        result = ZooDefs.Ids.OPEN_ACL_UNSAFE;
       }
       
       return result;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/solrj/src/java/org/apache/solr/common/cloud/ZkClientConnectionStrategy.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkClientConnectionStrategy.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkClientConnectionStrategy.java
index 43a2c52..acc5abf 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkClientConnectionStrategy.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkClientConnectionStrategy.java
@@ -97,7 +97,9 @@ public abstract class ZkClientConnectionStrategy {
   public boolean hasZkCredentialsToAddAutomatically() {
     return zkCredentialsToAddAutomatically != null;
   }
-  
+
+  public ZkCredentialsProvider getZkCredentialsToAddAutomatically() { return zkCredentialsToAddAutomatically; }
+
   protected SolrZooKeeper createSolrZooKeeper(final String serverAddress, final int zkClientTimeout,
       final Watcher watcher) throws IOException {
     SolrZooKeeper result = new SolrZooKeeper(serverAddress, zkClientTimeout, watcher);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestDelegationTokenRequest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestDelegationTokenRequest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestDelegationTokenRequest.java
new file mode 100644
index 0000000..47b8385
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestDelegationTokenRequest.java
@@ -0,0 +1,70 @@
+/*
+ * 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.request;
+
+import org.apache.lucene.util.LuceneTestCase;
+
+import org.junit.Test;
+
+/**
+ * Test for DelegationTokenRequests
+ */
+public class TestDelegationTokenRequest extends LuceneTestCase {
+
+  @Test
+  public void testGetRequest() throws Exception {
+    // without renewer
+    DelegationTokenRequest.Get get = new DelegationTokenRequest.Get();
+    assertEquals("GETDELEGATIONTOKEN", get.getParams().get("op"));
+    assertNull(get.getParams().get("renewer"));
+
+
+    // with renewer
+    final String renewer = "test";
+    get = new DelegationTokenRequest.Get(renewer);
+    assertEquals("GETDELEGATIONTOKEN", get.getParams().get("op"));
+    assertEquals(renewer, get.getParams().get("renewer"));
+  }
+
+  @Test
+  public void testRenewRequest() throws Exception {
+    final String token = "testToken";
+    DelegationTokenRequest.Renew renew = new DelegationTokenRequest.Renew(token);
+    assertEquals("RENEWDELEGATIONTOKEN", renew.getParams().get("op"));
+    assertEquals(token, renew.getParams().get("token"));
+    assertTrue(renew.getQueryParams().contains("op"));
+    assertTrue(renew.getQueryParams().contains("token"));
+
+    // can handle null token
+    renew = new DelegationTokenRequest.Renew(null);
+    renew.getParams();
+  }
+
+  @Test
+  public void testCancelRequest() throws Exception {
+    final String token = "testToken";
+    DelegationTokenRequest.Cancel cancel = new DelegationTokenRequest.Cancel(token);
+    assertEquals("CANCELDELEGATIONTOKEN", cancel.getParams().get("op"));
+    assertEquals(token, cancel.getParams().get("token"));
+    assertTrue(cancel.getQueryParams().contains("op"));
+    assertTrue(cancel.getQueryParams().contains("token"));
+
+    // can handle null token
+    cancel = new DelegationTokenRequest.Cancel(null);
+    cancel.getParams();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/solrj/src/test/org/apache/solr/client/solrj/response/TestDelegationTokenResponse.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/response/TestDelegationTokenResponse.java b/solr/solrj/src/test/org/apache/solr/client/solrj/response/TestDelegationTokenResponse.java
new file mode 100644
index 0000000..c376223
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/response/TestDelegationTokenResponse.java
@@ -0,0 +1,138 @@
+/*
+ * 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.response;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.commons.io.IOUtils;
+
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.client.solrj.ResponseParser;
+import org.apache.solr.client.solrj.request.DelegationTokenRequest;
+import org.apache.solr.common.SolrException;
+
+import org.junit.Test;
+
+import org.noggit.CharArr;
+import org.noggit.JSONWriter;
+
+public class TestDelegationTokenResponse extends LuceneTestCase {
+
+  private void delegationTokenResponse(DelegationTokenRequest request,
+      DelegationTokenResponse response, String responseBody) throws Exception {
+    ResponseParser parser = request.getResponseParser();
+    response.setResponse(parser.processResponse(
+      IOUtils.toInputStream(responseBody, "UTF-8"), "UTF-8"));
+  }
+
+  private String getNestedMapJson(String outerKey, String innerKey, Object innerValue) {
+    CharArr out = new CharArr();
+    JSONWriter w = new JSONWriter(out, 2);
+    Map<String, Object> innerMap = new HashMap<String, Object>();
+    innerMap.put(innerKey, innerValue);
+    Map<String, Map<String, Object>> outerMap = new HashMap<String, Map<String, Object>>();
+    outerMap.put(outerKey, innerMap);
+    w.write(outerMap);
+    return out.toString();
+  }
+
+  private String getMapJson(String key, Object value) {
+    CharArr out = new CharArr();
+    JSONWriter w = new JSONWriter(out, 2);
+    Map<String, Object> map = new HashMap<String, Object>();
+    map.put(key, value);
+    w.write(map);
+    return out.toString();
+  }
+
+  @Test
+  public void testGetResponse() throws Exception {
+    DelegationTokenRequest.Get getRequest = new DelegationTokenRequest.Get();
+    DelegationTokenResponse.Get getResponse = new DelegationTokenResponse.Get();
+
+    // not a map
+    try {
+      delegationTokenResponse(getRequest, getResponse, "");
+      getResponse.getDelegationToken();
+      fail("Expected SolrException");
+    } catch (SolrException se) {
+    }
+
+    // doesn't have Token outerMap
+    final String someToken = "someToken";
+    delegationTokenResponse(getRequest, getResponse, getNestedMapJson("NotToken", "urlString", someToken));
+    assertNull(getResponse.getDelegationToken());
+
+    // Token is not a map
+    try {
+      delegationTokenResponse(getRequest, getResponse, getMapJson("Token", someToken));
+      getResponse.getDelegationToken();
+      fail("Expected SolrException");
+    } catch (SolrException se) {
+    }
+
+    // doesn't have urlString
+    delegationTokenResponse(getRequest, getResponse, getNestedMapJson("Token", "notUrlString", someToken));
+    assertNull(getResponse.getDelegationToken());
+
+    // has Token + urlString
+    delegationTokenResponse(getRequest, getResponse, getNestedMapJson("Token", "urlString", someToken));
+    assertEquals(someToken, getResponse.getDelegationToken());
+  }
+
+  @Test
+  public void testRenewResponse() throws Exception {
+    DelegationTokenRequest.Renew renewRequest = new DelegationTokenRequest.Renew("token");
+    DelegationTokenResponse.Renew renewResponse = new DelegationTokenResponse.Renew();
+
+    // not a map
+    try {
+      delegationTokenResponse(renewRequest, renewResponse, "");
+      renewResponse.getExpirationTime();
+      fail("Expected SolrException");
+    } catch (SolrException se) {
+    }
+
+    // doesn't have long
+    delegationTokenResponse(renewRequest, renewResponse, getMapJson("notLong", "123"));
+    assertNull(renewResponse.getExpirationTime());
+
+    // long isn't valid
+    try {
+      delegationTokenResponse(renewRequest, renewResponse, getMapJson("long", "aaa"));
+      renewResponse.getExpirationTime();
+      fail("Expected SolrException");
+    } catch (SolrException se) {
+    }
+
+    // valid
+    Long expirationTime = new Long(Long.MAX_VALUE);
+    delegationTokenResponse(renewRequest, renewResponse,
+      getMapJson("long", expirationTime));
+    assertEquals(expirationTime, renewResponse.getExpirationTime());
+  }
+
+  @Test
+  public void testCancelResponse() throws Exception {
+    // expect empty response
+    DelegationTokenRequest.Cancel cancelRequest = new DelegationTokenRequest.Cancel("token");
+    DelegationTokenResponse.Cancel cancelResponse = new DelegationTokenResponse.Cancel();
+    delegationTokenResponse(cancelRequest, cancelResponse, "");
+  }
+}


[05/40] lucene-solr:apiv2: SOLR-9323: remove unused import (SQLHandler)

Posted by sa...@apache.org.
SOLR-9323: remove unused import (SQLHandler)


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

Branch: refs/heads/apiv2
Commit: f70adac1abb04b654f052a047ebe3b85b3c59e67
Parents: 0ad365c
Author: Christine Poerschke <cp...@apache.org>
Authored: Fri Jul 22 17:15:27 2016 +0100
Committer: Christine Poerschke <cp...@apache.org>
Committed: Fri Jul 22 17:15:27 2016 +0100

----------------------------------------------------------------------
 solr/core/src/java/org/apache/solr/handler/SQLHandler.java | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f70adac1/solr/core/src/java/org/apache/solr/handler/SQLHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/SQLHandler.java b/solr/core/src/java/org/apache/solr/handler/SQLHandler.java
index f83c43f..d273854 100644
--- a/solr/core/src/java/org/apache/solr/handler/SQLHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/SQLHandler.java
@@ -55,7 +55,6 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
 import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType;
 import org.apache.solr.client.solrj.io.stream.metrics.*;
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ModifiableSolrParams;


[32/40] lucene-solr:apiv2: SOLR-9200: Use direct package for Guava's Preconditions class

Posted by sa...@apache.org.
SOLR-9200: Use direct package for Guava's Preconditions class


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

Branch: refs/heads/apiv2
Commit: cead204fb6fecd576e8e1444e0de879d24d42f1b
Parents: 1fb4c5d
Author: Steve Rowe <sa...@gmail.com>
Authored: Thu Jul 28 19:18:04 2016 -0400
Committer: Steve Rowe <sa...@gmail.com>
Committed: Thu Jul 28 19:18:04 2016 -0400

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


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cead204f/solr/core/src/test/org/apache/solr/cloud/KerberosTestServices.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/KerberosTestServices.java b/solr/core/src/test/org/apache/solr/cloud/KerberosTestServices.java
index eb27218..1d7a4ab 100644
--- a/solr/core/src/test/org/apache/solr/cloud/KerberosTestServices.java
+++ b/solr/core/src/test/org/apache/solr/cloud/KerberosTestServices.java
@@ -26,7 +26,7 @@ import java.util.Locale;
 import java.util.Map;
 import java.util.Properties;
 
-import com.carrotsearch.ant.tasks.junit4.dependencies.com.google.common.base.Preconditions;
+import com.google.common.base.Preconditions;
 import org.apache.hadoop.minikdc.MiniKdc;
 import org.apache.solr.client.solrj.impl.Krb5HttpClientBuilder;
 


[15/40] lucene-solr:apiv2: LUCENE-7390: improve points indexing performance by letting the codec use transient heap in proportion to IndexWriter's indexing buffer, by default

Posted by sa...@apache.org.
LUCENE-7390: improve points indexing performance by letting the codec use transient heap in proportion to IndexWriter's indexing buffer, by default


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

Branch: refs/heads/apiv2
Commit: 3a0a9fd2c25965db4bde20975d9e6df4b540588a
Parents: 4ed68bc
Author: Mike McCandless <mi...@apache.org>
Authored: Mon Jul 25 11:33:34 2016 -0400
Committer: Mike McCandless <mi...@apache.org>
Committed: Mon Jul 25 11:33:34 2016 -0400

----------------------------------------------------------------------
 lucene/CHANGES.txt                               |  5 +++++
 .../simpletext/SimpleTextPointsWriter.java       |  4 ++--
 .../org/apache/lucene/codecs/PointsWriter.java   | 11 ++++++++---
 .../codecs/lucene60/Lucene60PointsWriter.java    | 15 ++++++++-------
 .../lucene/index/DocumentsWriterPerThread.java   |  2 +-
 .../org/apache/lucene/index/IndexWriter.java     |  2 +-
 .../lucene/index/LiveIndexWriterConfig.java      | 19 ++++++++-----------
 .../apache/lucene/index/PointValuesWriter.java   |  6 +++++-
 .../org/apache/lucene/util/bkd/BKDWriter.java    |  2 +-
 .../lucene60/TestLucene60PointsFormat.java       |  5 ++---
 .../apache/lucene/search/TestPointQueries.java   |  5 ++---
 .../apache/lucene/spatial3d/TestGeo3DPoint.java  |  5 ++---
 .../codecs/asserting/AssertingPointsFormat.java  |  4 ++--
 .../lucene/codecs/cranky/CrankyPointsFormat.java |  4 ++--
 .../apache/lucene/geo/BaseGeoPointTestCase.java  |  3 +--
 .../org/apache/lucene/index/RandomCodec.java     |  9 +++------
 16 files changed, 53 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a0a9fd2/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 7b53017..917dfa2 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -135,6 +135,11 @@ Improvements
 
 * LUCENE-7385: Improve/fix assert messages in SpanScorer. (David Smiley)
 
+* LUCENE-7390: Improve performance of indexing points by allowing the
+  codec to use transient heap in proportion to IndexWriter's RAM
+  buffer, instead of a fixed 16.0 MB.  A custom codec can still
+  override the buffer size itself. (Mike McCandless)
+
 Optimizations
 
 * LUCENE-7330, LUCENE-7339: Speed up conjunction queries. (Adrien Grand)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a0a9fd2/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsWriter.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsWriter.java
index 8d5c034..67289b6 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsWriter.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsWriter.java
@@ -68,7 +68,7 @@ class SimpleTextPointsWriter extends PointsWriter {
   }
 
   @Override
-  public void writeField(FieldInfo fieldInfo, PointsReader values) throws IOException {
+  public void writeField(FieldInfo fieldInfo, PointsReader values, double maxMBSortInHeap) throws IOException {
 
     boolean singleValuePerDoc = values.size(fieldInfo.name) == values.getDocCount(fieldInfo.name);
 
@@ -79,7 +79,7 @@ class SimpleTextPointsWriter extends PointsWriter {
                                           fieldInfo.getPointDimensionCount(),
                                           fieldInfo.getPointNumBytes(),
                                           BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE,
-                                          BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP,
+                                          maxMBSortInHeap,
                                           values.size(fieldInfo.name),
                                           singleValuePerDoc) {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a0a9fd2/lucene/core/src/java/org/apache/lucene/codecs/PointsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/PointsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/PointsWriter.java
index 05084db..8458497 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/PointsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/PointsWriter.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.MergeState;
+import org.apache.lucene.util.bkd.BKDWriter;
 
 /** Abstract API to write points
  *
@@ -34,8 +35,9 @@ public abstract class PointsWriter implements Closeable {
   protected PointsWriter() {
   }
 
-  /** Write all values contained in the provided reader */
-  public abstract void writeField(FieldInfo fieldInfo, PointsReader values) throws IOException;
+  /** Write all values contained in the provided reader.  {@code maxMBSortInHeap} is the maximum
+   *  transient heap that can be used to sort values, before spilling to disk for offline sorting */
+  public abstract void writeField(FieldInfo fieldInfo, PointsReader values, double maxMBSortInHeap) throws IOException;
 
   /** Default naive merge implementation for one field: it just re-indexes all the values
    *  from the incoming segment.  The default codec overrides this for 1D fields and uses
@@ -145,7 +147,10 @@ public abstract class PointsWriter implements Closeable {
                  public int getDocCount(String fieldName) {
                    return finalDocCount;
                  }
-               });
+               },
+               // TODO: also let merging of > 1D fields tap into IW's indexing buffer size, somehow (1D fields do an optimized merge sort
+               // and don't need heap)
+               BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP);
   }
 
   /** Default merge implementation to merge incoming points readers by visiting all their points and

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a0a9fd2/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java
index 63308c4..3acfac3 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java
@@ -39,7 +39,9 @@ import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.bkd.BKDReader;
 import org.apache.lucene.util.bkd.BKDWriter;
 
-/** Writes dimensional values */
+/** Writes dimensional values
+ *
+ * @lucene.experimental */
 public class Lucene60PointsWriter extends PointsWriter implements Closeable {
 
   /** Output used to write the BKD tree data file */
@@ -50,15 +52,13 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
 
   final SegmentWriteState writeState;
   final int maxPointsInLeafNode;
-  final double maxMBSortInHeap;
   private boolean finished;
 
   /** Full constructor */
-  public Lucene60PointsWriter(SegmentWriteState writeState, int maxPointsInLeafNode, double maxMBSortInHeap) throws IOException {
+  public Lucene60PointsWriter(SegmentWriteState writeState, int maxPointsInLeafNode) throws IOException {
     assert writeState.fieldInfos.hasPointValues();
     this.writeState = writeState;
     this.maxPointsInLeafNode = maxPointsInLeafNode;
-    this.maxMBSortInHeap = maxMBSortInHeap;
     String dataFileName = IndexFileNames.segmentFileName(writeState.segmentInfo.name,
                                                          writeState.segmentSuffix,
                                                          Lucene60PointsFormat.DATA_EXTENSION);
@@ -80,11 +80,11 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
 
   /** Uses the defaults values for {@code maxPointsInLeafNode} (1024) and {@code maxMBSortInHeap} (16.0) */
   public Lucene60PointsWriter(SegmentWriteState writeState) throws IOException {
-    this(writeState, BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE, BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP);
+    this(writeState, BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE);
   }
 
   @Override
-  public void writeField(FieldInfo fieldInfo, PointsReader values) throws IOException {
+  public void writeField(FieldInfo fieldInfo, PointsReader values, double maxMBSortInHeap) throws IOException {
 
     boolean singleValuePerDoc = values.size(fieldInfo.name) == values.getDocCount(fieldInfo.name);
 
@@ -173,7 +173,8 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
                                                 fieldInfo.getPointDimensionCount(),
                                                 fieldInfo.getPointNumBytes(),
                                                 maxPointsInLeafNode,
-                                                maxMBSortInHeap,
+                                                // NOTE: not used, since BKDWriter.merge does a merge sort:
+                                                BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP,
                                                 totMaxSize,
                                                 singleValuePerDoc)) {
             List<BKDReader> bkdReaders = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a0a9fd2/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 e72145c..351235e 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
@@ -153,7 +153,7 @@ class DocumentsWriterPerThread {
   final Allocator byteBlockAllocator;
   final IntBlockPool.Allocator intBlockAllocator;
   private final AtomicLong pendingNumDocs;
-  private final LiveIndexWriterConfig indexWriterConfig;
+  final LiveIndexWriterConfig indexWriterConfig;
   private final boolean enableTestPoints;
   private final IndexWriter indexWriter;
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a0a9fd2/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
index 0fb23d9..bdd9552 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -762,7 +762,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
    * {@link #getConfig()}.
    * 
    * <p>
-   * <b>NOTE:</b> after ths writer is created, the given configuration instance
+   * <b>NOTE:</b> after this writer is created, the given configuration instance
    * cannot be passed to another writer.
    * 
    * @param d

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a0a9fd2/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java b/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java
index cec70c0..fe4924d 100644
--- a/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java
+++ b/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java
@@ -168,9 +168,14 @@ public class LiveIndexWriterConfig {
   
   /**
    * Determines the amount of RAM that may be used for buffering added documents
-   * and deletions before they are flushed to the Directory. Generally for
-   * faster indexing performance it's best to flush by RAM usage instead of
-   * document count and use as large a RAM buffer as you can.
+   * and deletions before beginning to flush them to the Directory.  For
+   * faster indexing performance it's best to use as large a RAM buffer as you can.
+   * <p>
+   * Note that this setting is not a hard limit on memory usage during indexing, as
+   * transient and non-trivial memory well beyond this buffer size may be used,
+   * for example due to segment merges or writing points to new segments.
+   * For application stability the available memory in the JVM
+   * should be significantly larger than the RAM buffer used for indexing.
    * <p>
    * When this is set, the writer will flush whenever buffered documents and
    * deletions use this much RAM. Pass in
@@ -178,14 +183,6 @@ public class LiveIndexWriterConfig {
    * due to RAM usage. Note that if flushing by document count is also enabled,
    * then the flush will be triggered by whichever comes first.
    * <p>
-   * The maximum RAM limit is inherently determined by the JVMs available
-   * memory. Yet, an {@link IndexWriter} session can consume a significantly
-   * larger amount of memory than the given RAM limit since this limit is just
-   * an indicator when to flush memory resident documents to the Directory.
-   * Flushes are likely happen concurrently while other threads adding documents
-   * to the writer. For application stability the available memory in the JVM
-   * should be significantly larger than the RAM buffer used for indexing.
-   * <p>
    * <b>NOTE</b>: the account of RAM usage for pending deletions is only
    * approximate. Specifically, if you delete by Query, Lucene currently has no
    * way to measure the RAM usage of individual Queries so the accounting will

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a0a9fd2/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java b/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
index e3d1c1d..511635c 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
@@ -24,6 +24,7 @@ import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.Counter;
+import org.apache.lucene.util.bkd.BKDWriter;
 
 /** Buffers up pending byte[][] value(s) per doc, then flushes when segment flushes. */
 class PointValuesWriter {
@@ -35,6 +36,7 @@ class PointValuesWriter {
   private int numDocs;
   private int lastDocID = -1;
   private final byte[] packedValue;
+  private final LiveIndexWriterConfig indexWriterConfig;
 
   public PointValuesWriter(DocumentsWriterPerThread docWriter, FieldInfo fieldInfo) {
     this.fieldInfo = fieldInfo;
@@ -43,6 +45,7 @@ class PointValuesWriter {
     docIDs = new int[16];
     iwBytesUsed.addAndGet(16 * Integer.BYTES);
     packedValue = new byte[fieldInfo.getPointDimensionCount() * fieldInfo.getPointNumBytes()];
+    indexWriterConfig = docWriter.indexWriterConfig;
   }
 
   // TODO: if exactly the same value is added to exactly the same doc, should we dedup?
@@ -124,6 +127,7 @@ class PointValuesWriter {
                         public int getDocCount(String fieldName) {
                           return numDocs;
                         }
-                      });
+                      },
+                      Math.max(indexWriterConfig.getRAMBufferSizeMB()/8.0, BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP));
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a0a9fd2/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
index 09e6412..97651e4 100644
--- a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
@@ -204,7 +204,7 @@ public class BKDWriter implements Closeable {
     // all recursive halves (i.e. 16 + 8 + 4 + 2) so the memory usage is 2X
     // what that level would consume, so we multiply by 0.5 to convert from
     // bytes to points here.  Each dimension has its own sorted partition, so
-    // we must divide by numDims as wel.
+    // we must divide by numDims as well.
 
     maxPointsSortInHeap = (int) (0.5 * (maxMBSortInHeap * 1024 * 1024) / (bytesPerDoc * numDims));
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a0a9fd2/lucene/core/src/test/org/apache/lucene/codecs/lucene60/TestLucene60PointsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene60/TestLucene60PointsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene60/TestLucene60PointsFormat.java
index afa8ec4..4b898c3 100644
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene60/TestLucene60PointsFormat.java
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene60/TestLucene60PointsFormat.java
@@ -41,9 +41,8 @@ public class TestLucene60PointsFormat extends BasePointsFormatTestCase {
     if (random().nextBoolean()) {
       // randomize parameters
       int maxPointsInLeafNode = TestUtil.nextInt(random(), 50, 500);
-      double maxMBSortInHeap = 3.0 + (3*random().nextDouble());
       if (VERBOSE) {
-        System.out.println("TEST: using Lucene60PointsFormat with maxPointsInLeafNode=" + maxPointsInLeafNode + " and maxMBSortInHeap=" + maxMBSortInHeap);
+        System.out.println("TEST: using Lucene60PointsFormat with maxPointsInLeafNode=" + maxPointsInLeafNode);
       }
 
       // sneaky impersonation!
@@ -53,7 +52,7 @@ public class TestLucene60PointsFormat extends BasePointsFormatTestCase {
           return new PointsFormat() {
             @Override
             public PointsWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
-              return new Lucene60PointsWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap);
+              return new Lucene60PointsWriter(writeState, maxPointsInLeafNode);
             }
 
             @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a0a9fd2/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java b/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
index cf8372d..c139b64 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
@@ -1156,9 +1156,8 @@ public class TestPointQueries extends LuceneTestCase {
   private static Codec getCodec() {
     if (Codec.getDefault().getName().equals("Lucene62")) {
       int maxPointsInLeafNode = TestUtil.nextInt(random(), 16, 2048);
-      double maxMBSortInHeap = 5.0 + (3*random().nextDouble());
       if (VERBOSE) {
-        System.out.println("TEST: using Lucene60PointsFormat with maxPointsInLeafNode=" + maxPointsInLeafNode + " and maxMBSortInHeap=" + maxMBSortInHeap);
+        System.out.println("TEST: using Lucene60PointsFormat with maxPointsInLeafNode=" + maxPointsInLeafNode);
       }
 
       return new FilterCodec("Lucene62", Codec.getDefault()) {
@@ -1167,7 +1166,7 @@ public class TestPointQueries extends LuceneTestCase {
           return new PointsFormat() {
             @Override
             public PointsWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
-              return new Lucene60PointsWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap);
+              return new Lucene60PointsWriter(writeState, maxPointsInLeafNode);
             }
 
             @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a0a9fd2/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/TestGeo3DPoint.java
----------------------------------------------------------------------
diff --git a/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/TestGeo3DPoint.java b/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/TestGeo3DPoint.java
index d9baf61..bb98145 100644
--- a/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/TestGeo3DPoint.java
+++ b/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/TestGeo3DPoint.java
@@ -87,9 +87,8 @@ public class TestGeo3DPoint extends LuceneTestCase {
   private static Codec getCodec() {
     if (Codec.getDefault().getName().equals("Lucene62")) {
       int maxPointsInLeafNode = TestUtil.nextInt(random(), 16, 2048);
-      double maxMBSortInHeap = 3.0 + (3*random().nextDouble());
       if (VERBOSE) {
-        System.out.println("TEST: using Lucene60PointsFormat with maxPointsInLeafNode=" + maxPointsInLeafNode + " and maxMBSortInHeap=" + maxMBSortInHeap);
+        System.out.println("TEST: using Lucene60PointsFormat with maxPointsInLeafNode=" + maxPointsInLeafNode);
       }
 
       return new FilterCodec("Lucene62", Codec.getDefault()) {
@@ -98,7 +97,7 @@ public class TestGeo3DPoint extends LuceneTestCase {
           return new PointsFormat() {
             @Override
             public PointsWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
-              return new Lucene60PointsWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap);
+              return new Lucene60PointsWriter(writeState, maxPointsInLeafNode);
             }
 
             @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a0a9fd2/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java
index 731aaec..c3c672b 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java
@@ -254,11 +254,11 @@ public final class AssertingPointsFormat extends PointsFormat {
     }
     
     @Override
-    public void writeField(FieldInfo fieldInfo, PointsReader values) throws IOException {
+    public void writeField(FieldInfo fieldInfo, PointsReader values, double maxMBSortInHeap) throws IOException {
       if (fieldInfo.getPointDimensionCount() == 0) {
         throw new IllegalArgumentException("writing field=\"" + fieldInfo.name + "\" but pointDimensionalCount is 0");
       }
-      in.writeField(fieldInfo, values);
+      in.writeField(fieldInfo, values, maxMBSortInHeap);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a0a9fd2/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyPointsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyPointsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyPointsFormat.java
index fd2260b..ffd9a8c 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyPointsFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyPointsFormat.java
@@ -56,11 +56,11 @@ class CrankyPointsFormat extends PointsFormat {
     }
 
     @Override
-    public void writeField(FieldInfo fieldInfo, PointsReader values) throws IOException {
+    public void writeField(FieldInfo fieldInfo, PointsReader values, double maxMBSortInHeap) throws IOException {
       if (random.nextInt(100) == 0) {
         throw new IOException("Fake IOException");
       }  
-      delegate.writeField(fieldInfo, values);
+      delegate.writeField(fieldInfo, values, maxMBSortInHeap);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a0a9fd2/lucene/test-framework/src/java/org/apache/lucene/geo/BaseGeoPointTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/geo/BaseGeoPointTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/geo/BaseGeoPointTestCase.java
index 275c186..926132f 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/geo/BaseGeoPointTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/geo/BaseGeoPointTestCase.java
@@ -67,7 +67,6 @@ import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.SloppyMath;
 import org.apache.lucene.util.TestUtil;
-import org.apache.lucene.util.bkd.BKDWriter;
 
 /**
  * Abstract class to do basic tests for a geospatial impl (high level
@@ -1248,7 +1247,7 @@ public abstract class BaseGeoPointTestCase extends LuceneTestCase {
         return new PointsFormat() {
           @Override
           public PointsWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
-            return new Lucene60PointsWriter(writeState, pointsInLeaf, BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP);
+            return new Lucene60PointsWriter(writeState, pointsInLeaf);
           }
   
           @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a0a9fd2/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java b/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
index 127549f..6e8f7bc 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
@@ -92,7 +92,6 @@ public class RandomCodec extends AssertingCodec {
   // which is less effective for testing.
   // TODO: improve how we randomize this...
   private final int maxPointsInLeafNode;
-  private final double maxMBSortInHeap;
   private final int bkdSplitRandomSeed;
 
   @Override
@@ -103,9 +102,9 @@ public class RandomCodec extends AssertingCodec {
 
         // Randomize how BKDWriter chooses its splis:
 
-        return new Lucene60PointsWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap) {
+        return new Lucene60PointsWriter(writeState, maxPointsInLeafNode) {
           @Override
-          public void writeField(FieldInfo fieldInfo, PointsReader values) throws IOException {
+          public void writeField(FieldInfo fieldInfo, PointsReader values, double maxMBSortInHeap) throws IOException {
 
             boolean singleValuePerDoc = values.size(fieldInfo.name) == values.getDocCount(fieldInfo.name);
 
@@ -185,7 +184,6 @@ public class RandomCodec extends AssertingCodec {
     int lowFreqCutoff = TestUtil.nextInt(random, 2, 100);
 
     maxPointsInLeafNode = TestUtil.nextInt(random, 16, 2048);
-    maxMBSortInHeap = 5.0 + (3*random.nextDouble());
     bkdSplitRandomSeed = random.nextInt();
 
     add(avoidCodecs,
@@ -253,8 +251,7 @@ public class RandomCodec extends AssertingCodec {
   public String toString() {
     return super.toString() + ": " + previousMappings.toString() +
            ", docValues:" + previousDVMappings.toString() +
-           ", maxPointsInLeafNode=" + maxPointsInLeafNode +
-           ", maxMBSortInHeap=" + maxMBSortInHeap;
+           ", maxPointsInLeafNode=" + maxPointsInLeafNode;
   }
 
   /** Just like {@link BKDWriter} except it evilly picks random ways to split cells on


[35/40] lucene-solr:apiv2: LUCENE-7396: Speed flush of points.

Posted by sa...@apache.org.
LUCENE-7396: Speed flush of points.


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

Branch: refs/heads/apiv2
Commit: 60975d2dfa0bf855220db6d9755b7e24c14a59bb
Parents: 96280d1
Author: Adrien Grand <jp...@gmail.com>
Authored: Mon Jul 25 17:36:01 2016 +0200
Committer: Adrien Grand <jp...@gmail.com>
Committed: Fri Jul 29 10:23:54 2016 +0200

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   2 +
 .../lucene/codecs/MutablePointsReader.java      |  39 ++
 .../codecs/lucene60/Lucene60PointsWriter.java   |   9 +
 .../apache/lucene/index/PointValuesWriter.java  | 161 +++---
 .../java/org/apache/lucene/util/ArrayUtil.java  |  71 +--
 .../org/apache/lucene/util/ByteBlockPool.java   |   8 +
 .../org/apache/lucene/util/IntroSelector.java   | 126 +++++
 .../org/apache/lucene/util/IntroSorter.java     |   7 +-
 .../org/apache/lucene/util/RadixSelector.java   | 202 ++++++++
 .../java/org/apache/lucene/util/Selector.java   |  41 ++
 .../org/apache/lucene/util/bkd/BKDWriter.java   | 489 ++++++++++++++-----
 .../util/bkd/MutablePointsReaderUtils.java      | 185 +++++++
 .../apache/lucene/util/TestByteBlockPool.java   |   8 +-
 .../apache/lucene/util/TestIntroSelector.java   |  86 ++++
 .../apache/lucene/util/TestRadixSelector.java   |  77 +++
 .../util/bkd/TestMutablePointsReaderUtils.java  | 251 ++++++++++
 16 files changed, 1532 insertions(+), 230 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/60975d2d/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index a685839..67e2683 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -161,6 +161,8 @@ Optimizations
 * LUCENE-7311: Cached term queries do not seek the terms dictionary anymore.
   (Adrien Grand)
 
+* LUCENE-7396: Faster flush of points. (Adrien Grand, Mike McCandless)
+
 Other
 
 * LUCENE-4787: Fixed some highlighting javadocs. (Michael Dodsworth via Adrien

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/60975d2d/lucene/core/src/java/org/apache/lucene/codecs/MutablePointsReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/MutablePointsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/MutablePointsReader.java
new file mode 100644
index 0000000..b6c328b
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/MutablePointsReader.java
@@ -0,0 +1,39 @@
+/*
+ * 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.codecs;
+
+/** {@link PointsReader} whose order of points can be changed.
+ *  This class is useful for codecs to optimize flush.
+ *  @lucene.internal */
+public abstract class MutablePointsReader extends PointsReader {
+
+  /** Sole constructor. */
+  protected MutablePointsReader() {}
+
+  /** Fill {@code packedValue} with the packed bytes of the i-th value. */
+  public abstract void getValue(int i, byte[] packedValue);
+
+  /** Get the k-th byte of the i-th value. */
+  public abstract byte getByteAt(int i, int k);
+
+  /** Return the doc ID of the i-th value. */
+  public abstract int getDocID(int i);
+
+  /** Swap the i-th and j-th values. */
+  public abstract void swap(int i, int j);
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/60975d2d/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java
index 3acfac3..5fedf64 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java
@@ -25,6 +25,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.MutablePointsReader;
 import org.apache.lucene.codecs.PointsReader;
 import org.apache.lucene.codecs.PointsWriter;
 import org.apache.lucene.index.FieldInfo;
@@ -98,6 +99,14 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
                                           values.size(fieldInfo.name),
                                           singleValuePerDoc)) {
 
+      if (values instanceof MutablePointsReader) {
+        final long fp = writer.writeField(dataOut, fieldInfo.name, (MutablePointsReader) values);
+        if (fp != -1) {
+          indexFPs.put(fieldInfo.name, fp);
+        }
+        return;
+      }
+
       values.intersect(fieldInfo.name, new IntersectVisitor() {
           @Override
           public void visit(int docID) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/60975d2d/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java b/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
index 511635c..b4decb6 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
@@ -18,6 +18,7 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 
+import org.apache.lucene.codecs.MutablePointsReader;
 import org.apache.lucene.codecs.PointsReader;
 import org.apache.lucene.codecs.PointsWriter;
 import org.apache.lucene.util.ArrayUtil;
@@ -35,7 +36,7 @@ class PointValuesWriter {
   private int numPoints;
   private int numDocs;
   private int lastDocID = -1;
-  private final byte[] packedValue;
+  private final int packedBytesLength;
   private final LiveIndexWriterConfig indexWriterConfig;
 
   public PointValuesWriter(DocumentsWriterPerThread docWriter, FieldInfo fieldInfo) {
@@ -44,7 +45,7 @@ class PointValuesWriter {
     this.bytes = new ByteBlockPool(docWriter.byteBlockAllocator);
     docIDs = new int[16];
     iwBytesUsed.addAndGet(16 * Integer.BYTES);
-    packedValue = new byte[fieldInfo.getPointDimensionCount() * fieldInfo.getPointNumBytes()];
+    packedBytesLength = fieldInfo.getPointDimensionCount() * fieldInfo.getPointNumBytes();
     indexWriterConfig = docWriter.indexWriterConfig;
   }
 
@@ -70,64 +71,102 @@ class PointValuesWriter {
   }
 
   public void flush(SegmentWriteState state, PointsWriter writer) throws IOException {
-
-    writer.writeField(fieldInfo,
-                      new PointsReader() {
-                        @Override
-                        public void intersect(String fieldName, IntersectVisitor visitor) throws IOException {
-                          if (fieldName.equals(fieldInfo.name) == false) {
-                            throw new IllegalArgumentException("fieldName must be the same");
-                          }
-                          for(int i=0;i<numPoints;i++) {
-                            bytes.readBytes(packedValue.length * i, packedValue, 0, packedValue.length);
-                            visitor.visit(docIDs[i], packedValue);
-                          }
-                        }
-
-                        @Override
-                        public void checkIntegrity() {
-                          throw new UnsupportedOperationException();
-                        }
-
-                        @Override
-                        public long ramBytesUsed() {
-                          return 0L;
-                        }
-
-                        @Override
-                        public void close() {
-                        }
-
-                        @Override
-                        public byte[] getMinPackedValue(String fieldName) {
-                          throw new UnsupportedOperationException();
-                        }
-
-                        @Override
-                        public byte[] getMaxPackedValue(String fieldName) {
-                          throw new UnsupportedOperationException();
-                        }
-
-                        @Override
-                        public int getNumDimensions(String fieldName) {
-                          throw new UnsupportedOperationException();
-                        }
-
-                        @Override
-                        public int getBytesPerDimension(String fieldName) {
-                          throw new UnsupportedOperationException();
-                        }
-
-                        @Override
-                        public long size(String fieldName) {
-                          return numPoints;
-                        }
-
-                        @Override
-                        public int getDocCount(String fieldName) {
-                          return numDocs;
-                        }
-                      },
-                      Math.max(indexWriterConfig.getRAMBufferSizeMB()/8.0, BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP));
+    PointsReader reader = new MutablePointsReader() {
+
+      final int[] ords = new int[numPoints];
+      {
+        for (int i = 0; i < numPoints; ++i) {
+          ords[i] = i;
+        }
+      }
+
+      @Override
+      public void intersect(String fieldName, IntersectVisitor visitor) throws IOException {
+        if (fieldName.equals(fieldInfo.name) == false) {
+          throw new IllegalArgumentException("fieldName must be the same");
+        }
+        final byte[] packedValue = new byte[packedBytesLength];
+        for(int i=0;i<numPoints;i++) {
+          getValue(i, packedValue);
+          visitor.visit(getDocID(i), packedValue);
+        }
+      }
+
+      @Override
+      public void checkIntegrity() {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public long ramBytesUsed() {
+        return 0L;
+      }
+
+      @Override
+      public void close() {
+      }
+
+      @Override
+      public byte[] getMinPackedValue(String fieldName) {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public byte[] getMaxPackedValue(String fieldName) {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public int getNumDimensions(String fieldName) {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public int getBytesPerDimension(String fieldName) {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public long size(String fieldName) {
+        if (fieldName.equals(fieldInfo.name) == false) {
+          throw new IllegalArgumentException("fieldName must be the same");
+        }
+        return numPoints;
+      }
+
+      @Override
+      public int getDocCount(String fieldName) {
+        if (fieldName.equals(fieldInfo.name) == false) {
+          throw new IllegalArgumentException("fieldName must be the same");
+        }
+        return numDocs;
+      }
+
+      @Override
+      public void swap(int i, int j) {
+        int tmp = ords[i];
+        ords[i] = ords[j];
+        ords[j] = tmp;
+      }
+
+      @Override
+      public int getDocID(int i) {
+        return docIDs[ords[i]];
+      }
+
+      @Override
+      public void getValue(int i, byte[] packedValue) {
+        final long offset = (long) packedBytesLength * ords[i];
+        bytes.readBytes(offset, packedValue, 0, packedBytesLength);
+      }
+
+      @Override
+      public byte getByteAt(int i, int k) {
+        final long offset = (long) packedBytesLength * ords[i] + k;
+        return bytes.readByte(offset);
+      }
+    };
+
+    writer.writeField(fieldInfo, reader, Math.max(indexWriterConfig.getRAMBufferSizeMB()/8.0, BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP));
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/60975d2d/lucene/core/src/java/org/apache/lucene/util/ArrayUtil.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/ArrayUtil.java b/lucene/core/src/java/org/apache/lucene/util/ArrayUtil.java
index 0e10450..3bc65ef 100644
--- a/lucene/core/src/java/org/apache/lucene/util/ArrayUtil.java
+++ b/lucene/core/src/java/org/apache/lucene/util/ArrayUtil.java
@@ -459,69 +459,26 @@ public final class ArrayUtil {
    *  greater than or equal to it.
    *  This runs in linear time on average and in {@code n log(n)} time in the
    *  worst case.*/
-  public static <T> void select(T[] arr, int from, int to, int k, Comparator<T> comparator) {
-    if (k < from) {
-      throw new IllegalArgumentException("k must be >= from");
-    }
-    if (k >= to) {
-      throw new IllegalArgumentException("k must be < to");
-    }
-    final int maxDepth = 2 * MathUtil.log(to - from, 2);
-    quickSelect(arr, from, to, k, comparator, maxDepth);
-  }
-
-  private static <T> void quickSelect(T[] arr, int from, int to, int k, Comparator<T> comparator, int maxDepth) {
-    assert from <= k;
-    assert k < to;
-    if (to - from == 1) {
-      return;
-    }
-    if (--maxDepth < 0) {
-      Arrays.sort(arr, from, to, comparator);
-      return;
-    }
-
-    final int mid = (from + to) >>> 1;
-    // heuristic: we use the median of the values at from, to-1 and mid as a pivot
-    if (comparator.compare(arr[from], arr[to - 1]) > 0) {
-      swap(arr, from, to - 1);
-    }
-    if (comparator.compare(arr[to - 1], arr[mid]) > 0) {
-      swap(arr, to - 1, mid);
-      if (comparator.compare(arr[from], arr[to - 1]) > 0) {
-        swap(arr, from, to - 1);
-      }
-    }
+  public static <T> void select(T[] arr, int from, int to, int k, Comparator<? super T> comparator) {
+    new IntroSelector() {
 
-    T pivot = arr[to - 1];
+      T pivot;
 
-    int left = from + 1;
-    int right = to - 2;
-
-    for (;;) {
-      while (comparator.compare(pivot, arr[left]) > 0) {
-        ++left;
+      @Override
+      protected void swap(int i, int j) {
+        ArrayUtil.swap(arr, i, j);
       }
 
-      while (left < right && comparator.compare(pivot, arr[right]) <= 0) {
-        --right;
+      @Override
+      protected void setPivot(int i) {
+        pivot = arr[i];
       }
 
-      if (left < right) {
-        swap(arr, left, right);
-        --right;
-      } else {
-        break;
+      @Override
+      protected int comparePivot(int j) {
+        return comparator.compare(pivot, arr[j]);
       }
-    }
-    swap(arr, left, to - 1);
-
-    if (left == k) {
-      return;
-    } else if (left < k) {
-      quickSelect(arr, left + 1, to, k, comparator, maxDepth);
-    } else {
-      quickSelect(arr, from, left, k, comparator, maxDepth);
-    }
+    }.select(from, to, k);
   }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/60975d2d/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java b/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java
index 6bb12bd..e202d63 100644
--- a/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java
+++ b/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java
@@ -378,5 +378,13 @@ public final class ByteBlockPool {
       }
     } while (true);
   }
+
+  /** Read a single byte at the given {@code offset}. */
+  public byte readByte(long offset) {
+    int bufferIndex = (int) (offset >> BYTE_BLOCK_SHIFT);
+    int pos = (int) (offset & BYTE_BLOCK_MASK);
+    byte[] buffer = buffers[bufferIndex];
+    return buffer[pos];
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/60975d2d/lucene/core/src/java/org/apache/lucene/util/IntroSelector.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/IntroSelector.java b/lucene/core/src/java/org/apache/lucene/util/IntroSelector.java
new file mode 100644
index 0000000..7898535
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/util/IntroSelector.java
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util;
+
+import java.util.Comparator;
+
+/** Implementation of the quick select algorithm.
+ *  <p>It uses the median of the first, middle and last values as a pivot and
+ *  falls back to a heap sort when the number of recursion levels exceeds
+ *  {@code 2 lg(n)}, as a consequence it runs in linear time on average and in
+ *  {@code n log(n)} time in the worst case.</p>
+ *  @lucene.internal */
+public abstract class IntroSelector extends Selector {
+
+  @Override
+  public final void select(int from, int to, int k) {
+    checkArgs(from, to, k);
+    final int maxDepth = 2 * MathUtil.log(to - from, 2);
+    quickSelect(from, to, k, maxDepth);
+  }
+
+  // heap sort
+  void slowSelect(int from, int to, int k) {
+    new Sorter() {
+
+      @Override
+      protected void swap(int i, int j) {
+        IntroSelector.this.swap(i, j);
+      }
+
+      @Override
+      protected int compare(int i, int j) {
+        return IntroSelector.this.compare(i, j);
+      }
+
+      public void sort(int from, int to) {
+        heapSort(from, to);
+      }
+    }.sort(from, to);
+  }
+
+  private void quickSelect(int from, int to, int k, int maxDepth) {
+    assert from <= k;
+    assert k < to;
+    if (to - from == 1) {
+      return;
+    }
+    if (--maxDepth < 0) {
+      slowSelect(from, to, k);
+      return;
+    }
+
+    final int mid = (from + to) >>> 1;
+    // heuristic: we use the median of the values at from, to-1 and mid as a pivot
+    if (compare(from, to - 1) > 0) {
+      swap(from, to - 1);
+    }
+    if (compare(to - 1, mid) > 0) {
+      swap(to - 1, mid);
+      if (compare(from, to - 1) > 0) {
+        swap(from, to - 1);
+      }
+    }
+
+    setPivot(to - 1);
+
+    int left = from + 1;
+    int right = to - 2;
+
+    for (;;) {
+      while (comparePivot(left) > 0) {
+        ++left;
+      }
+
+      while (left < right && comparePivot(right) <= 0) {
+        --right;
+      }
+
+      if (left < right) {
+        swap(left, right);
+        --right;
+      } else {
+        break;
+      }
+    }
+    swap(left, to - 1);
+
+    if (left == k) {
+      return;
+    } else if (left < k) {
+      quickSelect(left + 1, to, k, maxDepth);
+    } else {
+      quickSelect(from, left, k, maxDepth);
+    }
+  }
+
+  /** Compare entries found in slots <code>i</code> and <code>j</code>.
+   *  The contract for the returned value is the same as
+   *  {@link Comparator#compare(Object, Object)}. */
+  protected int compare(int i, int j) {
+    setPivot(i);
+    return comparePivot(j);
+  }
+
+  /** Save the value at slot <code>i</code> so that it can later be used as a
+   * pivot, see {@link #comparePivot(int)}. */
+  protected abstract void setPivot(int i);
+
+  /** Compare the pivot with the slot at <code>j</code>, similarly to
+   *  {@link #compare(int, int) compare(i, j)}. */
+  protected abstract int comparePivot(int j);
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/60975d2d/lucene/core/src/java/org/apache/lucene/util/IntroSorter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/IntroSorter.java b/lucene/core/src/java/org/apache/lucene/util/IntroSorter.java
index 26f7e37..83d118d 100644
--- a/lucene/core/src/java/org/apache/lucene/util/IntroSorter.java
+++ b/lucene/core/src/java/org/apache/lucene/util/IntroSorter.java
@@ -16,7 +16,6 @@
  */
 package org.apache.lucene.util;
 
-
 /**
  * {@link Sorter} implementation based on a variant of the quicksort algorithm
  * called <a href="http://en.wikipedia.org/wiki/Introsort">introsort</a>: when
@@ -91,4 +90,10 @@ public abstract class IntroSorter extends Sorter {
   /** Compare the pivot with the slot at <code>j</code>, similarly to
    *  {@link #compare(int, int) compare(i, j)}. */
   protected abstract int comparePivot(int j);
+
+  @Override
+  protected int compare(int i, int j) {
+    setPivot(i);
+    return comparePivot(j);
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/60975d2d/lucene/core/src/java/org/apache/lucene/util/RadixSelector.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/RadixSelector.java b/lucene/core/src/java/org/apache/lucene/util/RadixSelector.java
new file mode 100644
index 0000000..543204a
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/util/RadixSelector.java
@@ -0,0 +1,202 @@
+/*
+ * 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;
+
+import java.util.Arrays;
+
+/** Radix selector.
+ *  <p>This implementation works similarly to a MSB radix sort except that it
+ *  only recurses into the sub partition that contains the desired value.
+ *  @lucene.internal */
+public abstract class RadixSelector extends Selector {
+
+  // after that many levels of recursion we fall back to introselect anyway
+  // this is used as a protection against the fact that radix sort performs
+  // worse when there are long common prefixes (probably because of cache
+  // locality)
+  private static final int LEVEL_THRESHOLD = 8;
+  // size of histograms: 256 + 1 to indicate that the string is finished
+  private static final int HISTOGRAM_SIZE = 257;
+  // buckets below this size will be sorted with introselect
+  private static final int LENGTH_THRESHOLD = 100;
+
+  // we store one histogram per recursion level
+  private final int[] histogram = new int[HISTOGRAM_SIZE];
+
+  private final int maxLength;
+
+  /**
+   * Sole constructor.
+   * @param maxLength the maximum length of keys, pass {@link Integer#MAX_VALUE} if unknown.
+   */
+  protected RadixSelector(int maxLength) {
+    this.maxLength = maxLength;
+  }
+
+  /** Return the k-th byte of the entry at index {@code i}, or {@code -1} if
+   * its length is less than or equal to {@code k}. This may only be called
+   * with a value of {@code i} between {@code 0} included and
+   * {@code maxLength} excluded. */
+  protected abstract int byteAt(int i, int k);
+
+  /** Get a fall-back selector which may assume that the first {@code d} bytes
+   *  of all compared strings are equal. This fallback selector is used when
+   *  the range becomes narrow or when the maximum level of recursion has
+   *  been exceeded. */
+  protected Selector getFallbackSelector(int d) {
+    return new IntroSelector() {
+      @Override
+      protected void swap(int i, int j) {
+        RadixSelector.this.swap(i, j);
+      }
+
+      @Override
+      protected int compare(int i, int j) {
+        for (int o = d; o < maxLength; ++o) {
+          final int b1 = byteAt(i, o);
+          final int b2 = byteAt(j, o);
+          if (b1 != b2) {
+            return b1 - b2;
+          } else if (b1 == -1) {
+            break;
+          }
+        }
+        return 0;
+      }
+
+      @Override
+      protected void setPivot(int i) {
+        pivot.setLength(0);
+        for (int o = d; o < maxLength; ++o) {
+          final int b = byteAt(i, o);
+          if (b == -1) {
+            break;
+          }
+          pivot.append((byte) b);
+        }
+      }
+
+      @Override
+      protected int comparePivot(int j) {
+        for (int o = 0; o < pivot.length(); ++o) {
+          final int b1 = pivot.byteAt(o) & 0xff;
+          final int b2 = byteAt(j, d + o);
+          if (b1 != b2) {
+            return b1 - b2;
+          }
+        }
+        if (d + pivot.length() == maxLength) {
+          return 0;
+        }
+        return -1 - byteAt(j, d + pivot.length());
+      }
+
+      private final BytesRefBuilder pivot = new BytesRefBuilder();
+    };
+  }
+
+  @Override
+  public void select(int from, int to, int k) {
+    checkArgs(from, to, k);
+    select(from, to, k, 0);
+  }
+
+  private void select(int from, int to, int k, int d) {
+    if (to - from <= LENGTH_THRESHOLD || d >= LEVEL_THRESHOLD) {
+      getFallbackSelector(d).select(from, to, k);
+    } else {
+      radixSelect(from, to, k, d);
+    }
+  }
+
+  private void radixSelect(int from, int to, int k, int d) {
+    final int[] histogram = this.histogram;
+    Arrays.fill(histogram, 0);
+
+    buildHistogram(from, to, d, histogram);
+
+    int bucketFrom = from;
+    for (int bucket = 0; bucket < HISTOGRAM_SIZE; ++bucket) {
+      final int bucketTo = bucketFrom + histogram[bucket];
+
+      if (bucketTo > k) {
+        partition(from, to, bucket, bucketFrom, bucketTo, d);
+
+        if (bucket != 0 && d + 1 < maxLength) {
+          // all elements in bucket 0 are equal so we only need to recurse if bucket != 0
+          select(bucketFrom, bucketTo, k, d + 1);
+        }
+        return;
+      }
+      bucketFrom = bucketTo;
+    }
+    throw new AssertionError("Unreachable code");
+  }
+
+  /** Return a number for the k-th character between 0 and {@link #HISTOGRAM_SIZE}. */
+  private int getBucket(int i, int k) {
+    return byteAt(i, k) + 1;
+  }
+
+  /** Build a histogram of the number of values per {@link #getBucket(int, int) bucket}. */
+  private int[] buildHistogram(int from, int to, int k, int[] histogram) {
+    for (int i = from; i < to; ++i) {
+      histogram[getBucket(i, k)]++;
+    }
+    return histogram;
+  }
+
+  /** Reorder elements so that all of them that fall into {@code bucket} are
+   *  between offsets {@code bucketFrom} and {@code bucketTo}. */
+  private void partition(int from, int to, int bucket, int bucketFrom, int bucketTo, int d) {
+    int left = from;
+    int right = to - 1;
+
+    int slot = bucketFrom;
+
+    for (;;) {
+      int leftBucket = getBucket(left, d);
+      int rightBucket = getBucket(right, d);
+
+      while (leftBucket <= bucket && left < bucketFrom) {
+        if (leftBucket == bucket) {
+          swap(left, slot++);
+        } else {
+          ++left;
+        }
+        leftBucket = getBucket(left, d);
+      }
+
+      while (rightBucket >= bucket && right >= bucketTo) {
+        if (rightBucket == bucket) {
+          swap(right, slot++);
+        } else {
+          --right;
+        }
+        rightBucket = getBucket(right, d);
+      }
+
+      if (left < bucketFrom && right >= bucketTo) {
+        swap(left++, right--);
+      } else {
+        assert left == bucketFrom;
+        assert right == bucketTo - 1;
+        break;
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/60975d2d/lucene/core/src/java/org/apache/lucene/util/Selector.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/Selector.java b/lucene/core/src/java/org/apache/lucene/util/Selector.java
new file mode 100644
index 0000000..8c17ce4
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/util/Selector.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util;
+
+/** An implementation of a selection algorithm, ie. computing the k-th greatest
+ *  value from a collection. */
+public abstract class Selector {
+
+  /** Reorder elements so that the element at position {@code k} is the same
+   *  as if all elements were sorted and all other elements are partitioned
+   *  around it: {@code [from, k)} only contains elements that are less than
+   *  or equal to {@code k} and {@code (k, to)} only contains elements that
+   *  are greater than or equal to {@code k}. */
+  public abstract void select(int from, int to, int k);
+
+  void checkArgs(int from, int to, int k) {
+    if (k < from) {
+      throw new IllegalArgumentException("k must be >= from");
+    }
+    if (k >= to) {
+      throw new IllegalArgumentException("k must be < to");
+    }
+  }
+
+  /** Swap values at slots <code>i</code> and <code>j</code>. */
+  protected abstract void swap(int i, int j);
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/60975d2d/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
index 97651e4..d0d7dca 100644
--- a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
@@ -25,6 +25,7 @@ import java.util.List;
 import java.util.function.IntFunction;
 
 import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.MutablePointsReader;
 import org.apache.lucene.index.MergeState;
 import org.apache.lucene.index.PointValues.IntersectVisitor;
 import org.apache.lucene.index.PointValues.Relation;
@@ -67,7 +68,7 @@ import org.apache.lucene.util.StringHelper;
  *  <p>
  *  See <a href="https://www.cs.duke.edu/~pankaj/publications/papers/bkd-sstd.pdf">this paper</a> for details.
  *
- *  <p>This consumes heap during writing: it allocates a <code>LongBitSet(numPoints)</code>, 
+ *  <p>This consumes heap during writing: it allocates a <code>LongBitSet(numPoints)</code>,
  *  and then uses up to the specified {@code maxMBSortInHeap} heap space for writing.
  *
  *  <p>
@@ -140,10 +141,10 @@ public class BKDWriter implements Closeable {
   /** True if every document has at most one value.  We specialize this case by not bothering to store the ord since it's redundant with docID.  */
   protected final boolean singleValuePerDoc;
 
-  /** How much heap OfflineSorter is allowed to use */ 
+  /** How much heap OfflineSorter is allowed to use */
   protected final OfflineSorter.BufferSize offlineSorterBufferMB;
 
-  /** How much heap OfflineSorter is allowed to use */ 
+  /** How much heap OfflineSorter is allowed to use */
   protected final int offlineSorterMaxTempFiles;
 
   private final int maxDoc;
@@ -381,7 +382,7 @@ public class BKDWriter implements Closeable {
         } else {
           mappedDocID = docMap.get(oldDocID);
         }
-        
+
         if (mappedDocID != -1) {
           // Not deleted!
           docID = mappedDocID;
@@ -416,15 +417,25 @@ public class BKDWriter implements Closeable {
     }
   }
 
-  /** More efficient bulk-add for incoming {@link BKDReader}s.  This does a merge sort of the already
-   *  sorted values and currently only works when numDims==1.  This returns -1 if all documents containing
-   *  dimensional values were deleted. */
-  public long merge(IndexOutput out, List<MergeState.DocMap> docMaps, List<BKDReader> readers) throws IOException {
-    if (numDims != 1) {
-      throw new UnsupportedOperationException("numDims must be 1 but got " + numDims);
+  /** Write a field from a {@link MutablePointsReader}. This way of writing
+   *  points is faster than regular writes with {@link BKDWriter#add} since
+   *  there is opportunity for reordering points before writing them to
+   *  disk. This method does not use transient disk in order to reorder points.
+   */
+  public long writeField(IndexOutput out, String fieldName, MutablePointsReader reader) throws IOException {
+    if (numDims == 1) {
+      return writeField1Dim(out, fieldName, reader);
+    } else {
+      return writeFieldNDims(out, fieldName, reader);
     }
+  }
+
+
+  /* In the 2+D case, we recursively pick the split dimension, compute the
+   * median value and partition other values around it. */
+  private long writeFieldNDims(IndexOutput out, String fieldName, MutablePointsReader reader) throws IOException {
     if (pointCount != 0) {
-      throw new IllegalStateException("cannot mix add and merge");
+      throw new IllegalStateException("cannot mix add and writeField");
     }
 
     // Catch user silliness:
@@ -435,6 +446,81 @@ public class BKDWriter implements Closeable {
     // Mark that we already finished:
     heapPointWriter = null;
 
+    long countPerLeaf = pointCount = reader.size(fieldName);
+    long innerNodeCount = 1;
+
+    while (countPerLeaf > maxPointsInLeafNode) {
+      countPerLeaf = (countPerLeaf+1)/2;
+      innerNodeCount *= 2;
+    }
+
+    int numLeaves = Math.toIntExact(innerNodeCount);
+
+    checkMaxLeafNodeCount(numLeaves);
+
+    final byte[] splitPackedValues = new byte[numLeaves * (bytesPerDim + 1)];
+    final long[] leafBlockFPs = new long[numLeaves];
+
+    // compute the min/max for this slice
+    Arrays.fill(minPackedValue, (byte) 0xff);
+    Arrays.fill(maxPackedValue, (byte) 0);
+    for (int i = 0; i < Math.toIntExact(pointCount); ++i) {
+      reader.getValue(i, scratch1);
+      for(int dim=0;dim<numDims;dim++) {
+        int offset = dim*bytesPerDim;
+        if (StringHelper.compare(bytesPerDim, scratch1, offset, minPackedValue, offset) < 0) {
+          System.arraycopy(scratch1, offset, minPackedValue, offset, bytesPerDim);
+        }
+        if (StringHelper.compare(bytesPerDim, scratch1, offset, maxPackedValue, offset) > 0) {
+          System.arraycopy(scratch1, offset, maxPackedValue, offset, bytesPerDim);
+        }
+      }
+
+      docsSeen.set(reader.getDocID(i));
+    }
+
+    build(1, numLeaves, reader, 0, Math.toIntExact(pointCount), out,
+        minPackedValue, maxPackedValue, splitPackedValues, leafBlockFPs,
+        new int[maxPointsInLeafNode]);
+
+    long indexFP = out.getFilePointer();
+    writeIndex(out, leafBlockFPs, splitPackedValues);
+    return indexFP;
+  }
+
+
+  /* In the 1D case, we can simply sort points in ascending order and use the
+   * same writing logic as we use at merge time. */
+  private long writeField1Dim(IndexOutput out, String fieldName, MutablePointsReader reader) throws IOException {
+    MutablePointsReaderUtils.sort(maxDoc, packedBytesLength, reader, 0, Math.toIntExact(reader.size(fieldName)));
+
+    final OneDimensionBKDWriter oneDimWriter = new OneDimensionBKDWriter(out);
+
+    reader.intersect(fieldName, new IntersectVisitor() {
+
+      @Override
+      public void visit(int docID, byte[] packedValue) throws IOException {
+        oneDimWriter.add(packedValue, docID);
+      }
+
+      @Override
+      public void visit(int docID) throws IOException {
+        throw new IllegalStateException();
+      }
+
+      @Override
+      public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+        return Relation.CELL_CROSSES_QUERY;
+      }
+    });
+
+    return oneDimWriter.finish();
+  }
+
+  /** More efficient bulk-add for incoming {@link BKDReader}s.  This does a merge sort of the already
+   *  sorted values and currently only works when numDims==1.  This returns -1 if all documents containing
+   *  dimensional values were deleted. */
+  public long merge(IndexOutput out, List<MergeState.DocMap> docMaps, List<BKDReader> readers) throws IOException {
     assert docMaps == null || readers.size() == docMaps.size();
 
     BKDMergeQueue queue = new BKDMergeQueue(bytesPerDim, readers.size());
@@ -453,126 +539,166 @@ public class BKDWriter implements Closeable {
       }
     }
 
-    if (queue.size() == 0) {
-      return -1;
+    OneDimensionBKDWriter oneDimWriter = new OneDimensionBKDWriter(out);
+
+    while (queue.size() != 0) {
+      MergeReader reader = queue.top();
+      // System.out.println("iter reader=" + reader);
+
+      // NOTE: doesn't work with subclasses (e.g. SimpleText!)
+      oneDimWriter.add(reader.state.scratchPackedValue, reader.docID);
+
+      if (reader.next()) {
+        queue.updateTop();
+      } else {
+        // This segment was exhausted
+        queue.pop();
+      }
     }
 
-    int leafCount = 0;
-    List<Long> leafBlockFPs = new ArrayList<>();
-    List<byte[]> leafBlockStartValues = new ArrayList<>();
+    return oneDimWriter.finish();
+  }
+
+  private class OneDimensionBKDWriter {
+
+    final IndexOutput out;
+    final int pointsPerLeafBlock = (int) (0.75 * maxPointsInLeafNode);
+    final List<Long> leafBlockFPs = new ArrayList<>();
+    final List<byte[]> leafBlockStartValues = new ArrayList<>();
+    final byte[] leafValues = new byte[pointsPerLeafBlock * packedBytesLength];
+    final int[] leafDocs = new int[pointsPerLeafBlock];
+    long valueCount;
+    int leafCount;
+
+    OneDimensionBKDWriter(IndexOutput out) {
+      if (numDims != 1) {
+        throw new UnsupportedOperationException("numDims must be 1 but got " + numDims);
+      }
+      if (pointCount != 0) {
+        throw new IllegalStateException("cannot mix add and merge");
+      }
+
+      // Catch user silliness:
+      if (heapPointWriter == null && tempInput == null) {
+        throw new IllegalStateException("already finished");
+      }
 
-    // Target halfway between min and max allowed for the leaf:
-    int pointsPerLeafBlock = (int) (0.75 * maxPointsInLeafNode);
-    //System.out.println("POINTS PER: " + pointsPerLeafBlock);
+      // Mark that we already finished:
+      heapPointWriter = null;
 
-    byte[] lastPackedValue = new byte[bytesPerDim];
-    byte[] firstPackedValue = new byte[bytesPerDim];
-    long valueCount = 0;
+      this.out = out;
 
-    // Buffer up each leaf block's docs and values
-    int[] leafBlockDocIDs = new int[maxPointsInLeafNode];
-    byte[][] leafBlockPackedValues = new byte[maxPointsInLeafNode][];
-    for(int i=0;i<maxPointsInLeafNode;i++) {
-      leafBlockPackedValues[i] = new byte[packedBytesLength];
+      lastPackedValue = new byte[packedBytesLength];
     }
-    Arrays.fill(commonPrefixLengths, bytesPerDim);
 
-    while (queue.size() != 0) {
-      MergeReader reader = queue.top();
-      // System.out.println("iter reader=" + reader);
+    // for asserts
+    final byte[] lastPackedValue;
+    int lastDocID;
 
-      // NOTE: doesn't work with subclasses (e.g. SimpleText!)
-      int docID = reader.docID;
-      leafBlockDocIDs[leafCount] = docID;
-      System.arraycopy(reader.state.scratchPackedValue, 0, leafBlockPackedValues[leafCount], 0, packedBytesLength);
-      docsSeen.set(docID);
+    void add(byte[] packedValue, int docID) throws IOException {
+      assert valueInOrder(valueCount + leafCount,
+          0, lastPackedValue, packedValue, 0, docID, lastDocID);
 
-      if (valueCount == 0) {
-        System.arraycopy(reader.state.scratchPackedValue, 0, minPackedValue, 0, packedBytesLength);
-      }
-      System.arraycopy(reader.state.scratchPackedValue, 0, maxPackedValue, 0, packedBytesLength);
+      System.arraycopy(packedValue, 0, leafValues, leafCount * packedBytesLength, packedBytesLength);
+      leafDocs[leafCount] = docID;
+      docsSeen.set(docID);
+      leafCount++;
 
-      assert numDims > 1 || valueInOrder(valueCount, lastPackedValue, reader.state.scratchPackedValue, 0);
-      valueCount++;
-      if (pointCount > totalPointCount) {
+      if (valueCount > totalPointCount) {
         throw new IllegalStateException("totalPointCount=" + totalPointCount + " was passed when we were created, but we just hit " + pointCount + " values");
       }
 
-      if (leafCount == 0) {
-        if (leafBlockFPs.size() > 0) {
-          // Save the first (minimum) value in each leaf block except the first, to build the split value index in the end:
-          leafBlockStartValues.add(Arrays.copyOf(reader.state.scratchPackedValue, bytesPerDim));
-        }
-        Arrays.fill(commonPrefixLengths, bytesPerDim);
-        System.arraycopy(reader.state.scratchPackedValue, 0, firstPackedValue, 0, bytesPerDim);
-      } else {
-        // Find per-dim common prefix:
-        for(int dim=0;dim<numDims;dim++) {
-          int offset = dim * bytesPerDim;
-          for(int j=0;j<commonPrefixLengths[dim];j++) {
-            if (firstPackedValue[offset+j] != reader.state.scratchPackedValue[offset+j]) {
-              commonPrefixLengths[dim] = j;
-              break;
-            }
-          }
-        }
+      if (leafCount == pointsPerLeafBlock) {
+        // We write a block once we hit exactly the max count ... this is different from
+        // when we flush a new segment, where we write between max/2 and max per leaf block,
+        // so merged segments will behave differently from newly flushed segments:
+        writeLeafBlock();
+        leafCount = 0;
       }
 
-      leafCount++;
+      assert (lastDocID = docID) >= 0; // only assign when asserts are enabled
+    }
 
-      if (reader.next()) {
-        queue.updateTop();
-      } else {
-        // This segment was exhausted
-        queue.pop();
+    public long finish() throws IOException {
+      if (leafCount > 0) {
+        writeLeafBlock();
+        leafCount = 0;
       }
 
-      // We write a block once we hit exactly the max count ... this is different from
-      // when we flush a new segment, where we write between max/2 and max per leaf block,
-      // so merged segments will behave differently from newly flushed segments:
-      if (leafCount == pointsPerLeafBlock || queue.size() == 0) {
-        leafBlockFPs.add(out.getFilePointer());
-        checkMaxLeafNodeCount(leafBlockFPs.size());
+      if (valueCount == 0) {
+        return -1;
+      }
 
-        writeLeafBlockDocs(out, leafBlockDocIDs, 0, leafCount);
-        writeCommonPrefixes(out, commonPrefixLengths, firstPackedValue);
+      pointCount = valueCount;
 
-        final IntFunction<BytesRef> packedValues = new IntFunction<BytesRef>() {
-          final BytesRef scratch = new BytesRef();
+      long indexFP = out.getFilePointer();
 
-          {
-            scratch.length = packedBytesLength;
-            scratch.offset = 0;
-          }
+      int numInnerNodes = leafBlockStartValues.size();
 
-          @Override
-          public BytesRef apply(int i) {
-            scratch.bytes = leafBlockPackedValues[i];
-            return scratch;
-          }
-        };
-        writeLeafBlockPackedValues(out, commonPrefixLengths, leafCount, 0, packedValues);
+      //System.out.println("BKDW: now rotate numInnerNodes=" + numInnerNodes + " leafBlockStarts=" + leafBlockStartValues.size());
 
-        leafCount = 0;
+      byte[] index = new byte[(1+numInnerNodes) * (1+bytesPerDim)];
+      rotateToTree(1, 0, numInnerNodes, index, leafBlockStartValues);
+      long[] arr = new long[leafBlockFPs.size()];
+      for(int i=0;i<leafBlockFPs.size();i++) {
+        arr[i] = leafBlockFPs.get(i);
       }
+      writeIndex(out, arr, index);
+      return indexFP;
     }
 
-    pointCount = valueCount;
+    private void writeLeafBlock() throws IOException {
+      assert leafCount != 0;
+      if (valueCount == 0) {
+        System.arraycopy(leafValues, 0, minPackedValue, 0, packedBytesLength);
+      }
+      System.arraycopy(leafValues, (leafCount - 1) * packedBytesLength, maxPackedValue, 0, packedBytesLength);
+
+      valueCount += leafCount;
 
-    long indexFP = out.getFilePointer();
+      if (leafBlockFPs.size() > 0) {
+        // Save the first (minimum) value in each leaf block except the first, to build the split value index in the end:
+        leafBlockStartValues.add(Arrays.copyOf(leafValues, packedBytesLength));
+      }
+      leafBlockFPs.add(out.getFilePointer());
+      checkMaxLeafNodeCount(leafBlockFPs.size());
+
+      Arrays.fill(commonPrefixLengths, bytesPerDim);
+      // Find per-dim common prefix:
+      for(int dim=0;dim<numDims;dim++) {
+        int offset1 = dim * bytesPerDim;
+        int offset2 = (leafCount - 1) * packedBytesLength + offset1;
+        for(int j=0;j<commonPrefixLengths[dim];j++) {
+          if (leafValues[offset1+j] != leafValues[offset2+j]) {
+            commonPrefixLengths[dim] = j;
+            break;
+          }
+        }
+      }
+
+      writeLeafBlockDocs(out, leafDocs, 0, leafCount);
+      writeCommonPrefixes(out, commonPrefixLengths, leafValues);
 
-    int numInnerNodes = leafBlockStartValues.size();
+      final IntFunction<BytesRef> packedValues = new IntFunction<BytesRef>() {
+        final BytesRef scratch = new BytesRef();
 
-    //System.out.println("BKDW: now rotate numInnerNodes=" + numInnerNodes + " leafBlockStarts=" + leafBlockStartValues.size());
+        {
+          scratch.length = packedBytesLength;
+          scratch.bytes = leafValues;
+        }
 
-    byte[] index = new byte[(1+numInnerNodes) * (1+bytesPerDim)];
-    rotateToTree(1, 0, numInnerNodes, index, leafBlockStartValues);
-    long[] arr = new long[leafBlockFPs.size()];
-    for(int i=0;i<leafBlockFPs.size();i++) {
-      arr[i] = leafBlockFPs.get(i);
+        @Override
+        public BytesRef apply(int i) {
+          scratch.offset = packedBytesLength * i;
+          return scratch;
+        }
+      };
+      assert valuesInOrderAndBounds(leafCount, 0, Arrays.copyOf(leafValues, packedBytesLength),
+          Arrays.copyOfRange(leafValues, (leafCount - 1) * packedBytesLength, leafCount * packedBytesLength),
+          packedValues, leafDocs, 0);
+      writeLeafBlockPackedValues(out, commonPrefixLengths, leafCount, 0, packedValues);
     }
-    writeIndex(out, arr, index);
-    return indexFP;
+
   }
 
   // TODO: there must be a simpler way?
@@ -937,7 +1063,7 @@ public class BKDWriter implements Closeable {
       int compressedByteOffset = sortedDim * bytesPerDim + commonPrefixLengths[sortedDim];
       commonPrefixLengths[sortedDim]++;
       for (int i = 0; i < count; ) {
-        // do run-length compression on the byte at compressedByteOffset 
+        // do run-length compression on the byte at compressedByteOffset
         int runLen = runLen(packedValues, i, Math.min(i + 0xff, count), compressedByteOffset);
         assert runLen <= 0xff;
         BytesRef first = packedValues.apply(i);
@@ -1016,7 +1142,7 @@ public class BKDWriter implements Closeable {
     }
   }
 
-  /** Called on exception, to check whether the checksum is also corrupt in this source, and add that 
+  /** Called on exception, to check whether the checksum is also corrupt in this source, and add that
    *  information (checksum matched or didn't) as a suppressed exception. */
   private void verifyChecksum(Throwable priorException, PointWriter writer) throws IOException {
     // TODO: we could improve this, to always validate checksum as we recurse, if we shared left and
@@ -1110,6 +1236,136 @@ public class BKDWriter implements Closeable {
     }
   }
 
+  /* Recursively reorders the provided reader and writes the bkd-tree on the fly. */
+  private void build(int nodeID, int leafNodeOffset,
+      MutablePointsReader reader, int from, int to,
+      IndexOutput out,
+      byte[] minPackedValue, byte[] maxPackedValue,
+      byte[] splitPackedValues,
+      long[] leafBlockFPs,
+      int[] spareDocIds) throws IOException {
+
+    if (nodeID >= leafNodeOffset) {
+      // leaf node
+      final int count = to - from;
+      assert count <= maxPointsInLeafNode;
+
+      // Compute common prefixes
+      Arrays.fill(commonPrefixLengths, bytesPerDim);
+      reader.getValue(from, scratch1);
+      for (int i = from + 1; i < to; ++i) {
+        reader.getValue(i, scratch2);
+        for (int dim=0;dim<numDims;dim++) {
+          final int offset = dim * bytesPerDim;
+          for(int j=0;j<commonPrefixLengths[dim];j++) {
+            if (scratch1[offset+j] != scratch2[offset+j]) {
+              commonPrefixLengths[dim] = j;
+              break;
+            }
+          }
+        }
+      }
+
+      // Find the dimension that has the least number of unique bytes at commonPrefixLengths[dim]
+      FixedBitSet[] usedBytes = new FixedBitSet[numDims];
+      for (int dim = 0; dim < numDims; ++dim) {
+        if (commonPrefixLengths[dim] < bytesPerDim) {
+          usedBytes[dim] = new FixedBitSet(256);
+        }
+      }
+      for (int i = from + 1; i < to; ++i) {
+        for (int dim=0;dim<numDims;dim++) {
+          if (usedBytes[dim] != null) {
+            byte b = reader.getByteAt(i, dim * bytesPerDim + commonPrefixLengths[dim]);
+            usedBytes[dim].set(Byte.toUnsignedInt(b));
+          }
+        }
+      }
+      int sortedDim = 0;
+      int sortedDimCardinality = Integer.MAX_VALUE;
+      for (int dim = 0; dim < numDims; ++dim) {
+        if (usedBytes[dim] != null) {
+          final int cardinality = usedBytes[dim].cardinality();
+          if (cardinality < sortedDimCardinality) {
+            sortedDim = dim;
+            sortedDimCardinality = cardinality;
+          }
+        }
+      }
+
+      // sort by sortedDim
+      MutablePointsReaderUtils.sortByDim(sortedDim, bytesPerDim, commonPrefixLengths,
+          reader, from, to, scratch1, scratch2);
+
+      // Save the block file pointer:
+      leafBlockFPs[nodeID - leafNodeOffset] = out.getFilePointer();
+
+      // Write doc IDs
+      int[] docIDs = spareDocIds;
+      for (int i = from; i < to; ++i) {
+        docIDs[i - from] = reader.getDocID(i);
+      }
+      writeLeafBlockDocs(out, docIDs, 0, count);
+
+      // Write the common prefixes:
+      reader.getValue(from, scratch1);
+      writeCommonPrefixes(out, commonPrefixLengths, scratch1);
+
+      // Write the full values:
+      IntFunction<BytesRef> packedValues = new IntFunction<BytesRef>() {
+        final BytesRef scratch = new BytesRef(packedBytesLength);
+
+        {
+          scratch.offset = 0;
+          scratch.length = packedBytesLength;
+        }
+
+        @Override
+        public BytesRef apply(int i) {
+          reader.getValue(from + i, scratch.bytes);
+          return scratch;
+        }
+      };
+      assert valuesInOrderAndBounds(count, sortedDim, minPackedValue, maxPackedValue, packedValues,
+          docIDs, 0);
+      writeLeafBlockPackedValues(out, commonPrefixLengths, count, sortedDim, packedValues);
+
+    } else {
+      // inner node
+
+      // compute the split dimension and partition around it
+      final int splitDim = split(minPackedValue, maxPackedValue);
+      final int mid = (from + to + 1) >>> 1;
+
+      int commonPrefixLen = bytesPerDim;
+      for (int i = 0; i < bytesPerDim; ++i) {
+        if (minPackedValue[splitDim * bytesPerDim + i] != maxPackedValue[splitDim * bytesPerDim + i]) {
+          commonPrefixLen = i;
+          break;
+        }
+      }
+      MutablePointsReaderUtils.partition(maxDoc, splitDim, bytesPerDim, commonPrefixLen,
+          reader, from, to, mid, scratch1, scratch2);
+
+      // set the split value
+      final int address = nodeID * (1+bytesPerDim);
+      splitPackedValues[address] = (byte) splitDim;
+      reader.getValue(mid, scratch1);
+      System.arraycopy(scratch1, splitDim * bytesPerDim, splitPackedValues, address + 1, bytesPerDim);
+
+      byte[] minSplitPackedValue = Arrays.copyOf(minPackedValue, packedBytesLength);
+      byte[] maxSplitPackedValue = Arrays.copyOf(maxPackedValue, packedBytesLength);
+      System.arraycopy(scratch1, splitDim * bytesPerDim, minSplitPackedValue, splitDim * bytesPerDim, bytesPerDim);
+      System.arraycopy(scratch1, splitDim * bytesPerDim, maxSplitPackedValue, splitDim * bytesPerDim, bytesPerDim);
+
+      // recurse
+      build(nodeID * 2, leafNodeOffset, reader, from, mid, out,
+          minPackedValue, maxSplitPackedValue, splitPackedValues, leafBlockFPs, spareDocIds);
+      build(nodeID * 2 + 1, leafNodeOffset, reader, mid, to, out,
+          minSplitPackedValue, maxPackedValue, splitPackedValues, leafBlockFPs, spareDocIds);
+    }
+  }
+
   /** The array (sized numDims) of PathSlice describe the cell we have currently recursed to. */
   private void build(int nodeID, int leafNodeOffset,
                      PathSlice[] slices,
@@ -1217,7 +1473,8 @@ public class BKDWriter implements Closeable {
           return scratch;
         }
       };
-      assert valuesInOrderAndBounds(count, minPackedValue, maxPackedValue, packedValues);
+      assert valuesInOrderAndBounds(count, sortedDim, minPackedValue, maxPackedValue, packedValues,
+          heapSource.docIDs, Math.toIntExact(source.start));
       writeLeafBlockPackedValues(out, commonPrefixLengths, count, sortedDim, packedValues);
 
     } else {
@@ -1321,12 +1578,16 @@ public class BKDWriter implements Closeable {
   }
 
   // only called from assert
-  private boolean valuesInOrderAndBounds(int count, byte[] minPackedValue, byte[] maxPackedValue, IntFunction<BytesRef> values) throws IOException {
-    byte[] lastPackedValue = new byte[bytesPerDim];
+  private boolean valuesInOrderAndBounds(int count, int sortedDim, byte[] minPackedValue, byte[] maxPackedValue,
+      IntFunction<BytesRef> values, int[] docs, int docsOffset) throws IOException {
+    byte[] lastPackedValue = new byte[packedBytesLength];
+    int lastDoc = -1;
     for (int i=0;i<count;i++) {
       BytesRef packedValue = values.apply(i);
       assert packedValue.length == packedBytesLength;
-      assert numDims != 1 || valueInOrder(i, lastPackedValue, packedValue.bytes, packedValue.offset);
+      assert valueInOrder(i, sortedDim, lastPackedValue, packedValue.bytes, packedValue.offset,
+          docs[docsOffset + i], lastDoc);
+      lastDoc = docs[docsOffset + i];
 
       // Make sure this value does in fact fall within this leaf cell:
       assert valueInBounds(packedValue, minPackedValue, maxPackedValue);
@@ -1335,11 +1596,19 @@ public class BKDWriter implements Closeable {
   }
 
   // only called from assert
-  private boolean valueInOrder(long ord, byte[] lastPackedValue, byte[] packedValue, int packedValueOffset) {
-    if (ord > 0 && StringHelper.compare(bytesPerDim, lastPackedValue, 0, packedValue, packedValueOffset) > 0) {
-      throw new AssertionError("values out of order: last value=" + new BytesRef(lastPackedValue) + " current value=" + new BytesRef(packedValue, packedValueOffset, packedBytesLength) + " ord=" + ord);
+  private boolean valueInOrder(long ord, int sortedDim, byte[] lastPackedValue, byte[] packedValue, int packedValueOffset,
+      int doc, int lastDoc) {
+    int dimOffset = sortedDim * bytesPerDim;
+    if (ord > 0) {
+      int cmp = StringHelper.compare(bytesPerDim, lastPackedValue, dimOffset, packedValue, packedValueOffset + dimOffset);
+      if (cmp > 0) {
+        throw new AssertionError("values out of order: last value=" + new BytesRef(lastPackedValue) + " current value=" + new BytesRef(packedValue, packedValueOffset, packedBytesLength) + " ord=" + ord);
+      }
+      if (cmp == 0 && doc < lastDoc) {
+        throw new AssertionError("docs out of order: last doc=" + lastDoc + " current doc=" + doc + " ord=" + ord);
+      }
     }
-    System.arraycopy(packedValue, packedValueOffset, lastPackedValue, 0, bytesPerDim);
+    System.arraycopy(packedValue, packedValueOffset, lastPackedValue, 0, packedBytesLength);
     return true;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/60975d2d/lucene/core/src/java/org/apache/lucene/util/bkd/MutablePointsReaderUtils.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/bkd/MutablePointsReaderUtils.java b/lucene/core/src/java/org/apache/lucene/util/bkd/MutablePointsReaderUtils.java
new file mode 100644
index 0000000..d499e2b
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/util/bkd/MutablePointsReaderUtils.java
@@ -0,0 +1,185 @@
+/*
+ * 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.bkd;
+
+import org.apache.lucene.codecs.MutablePointsReader;
+import org.apache.lucene.util.IntroSelector;
+import org.apache.lucene.util.IntroSorter;
+import org.apache.lucene.util.MSBRadixSorter;
+import org.apache.lucene.util.RadixSelector;
+import org.apache.lucene.util.Selector;
+import org.apache.lucene.util.StringHelper;
+import org.apache.lucene.util.packed.PackedInts;
+
+final class MutablePointsReaderUtils {
+
+  MutablePointsReaderUtils() {}
+
+  /** Sort the given {@link MutablePointsReader} based on its packed value then doc ID. */
+  static void sort(int maxDoc, int packedBytesLength,
+      MutablePointsReader reader, int from, int to) {
+    final int bitsPerDocId = PackedInts.bitsRequired(maxDoc - 1);
+    new MSBRadixSorter(packedBytesLength + (bitsPerDocId + 7) / 8) {
+
+      @Override
+      protected void swap(int i, int j) {
+        reader.swap(i, j);
+      }
+
+      @Override
+      protected int byteAt(int i, int k) {
+        if (k < packedBytesLength) {
+          return Byte.toUnsignedInt(reader.getByteAt(i, k));
+        } else {
+          final int shift = bitsPerDocId - ((k - packedBytesLength + 1) << 3);
+          return (reader.getDocID(i) >>> Math.max(0, shift)) & 0xff;
+        }
+      }
+
+      @Override
+      protected org.apache.lucene.util.Sorter getFallbackSorter(int k) {
+        return new IntroSorter() {
+
+          final byte[] pivot = new byte[packedBytesLength];
+          final byte[] scratch = new byte[packedBytesLength];
+          int pivotDoc;
+
+          @Override
+          protected void swap(int i, int j) {
+            reader.swap(i, j);
+          }
+
+          @Override
+          protected void setPivot(int i) {
+            reader.getValue(i, pivot);
+            pivotDoc = reader.getDocID(i);
+          }
+
+          @Override
+          protected int comparePivot(int j) {
+            if (k < packedBytesLength) {
+              reader.getValue(j, scratch);
+              int cmp = StringHelper.compare(packedBytesLength - k, pivot, k, scratch, k);
+              if (cmp != 0) {
+                return cmp;
+              }
+            }
+            return pivotDoc - reader.getDocID(j);
+          }
+        };
+      }
+
+    }.sort(from, to);
+  }
+
+  /** Sort points on the given dimension. */
+  static void sortByDim(int sortedDim, int bytesPerDim, int[] commonPrefixLengths,
+      MutablePointsReader reader, int from, int to,
+      byte[] scratch1, byte[] scratch2) {
+
+    // No need for a fancy radix sort here, this is called on the leaves only so
+    // there are not many values to sort
+    final int offset = sortedDim * bytesPerDim + commonPrefixLengths[sortedDim];
+    final int numBytesToCompare = bytesPerDim - commonPrefixLengths[sortedDim];
+    new IntroSorter() {
+
+      final byte[] pivot = scratch1;
+      int pivotDoc = -1;
+
+      @Override
+      protected void swap(int i, int j) {
+        reader.swap(i, j);
+      }
+
+      @Override
+      protected void setPivot(int i) {
+        reader.getValue(i, pivot);
+        pivotDoc = reader.getDocID(i);
+      }
+
+      @Override
+      protected int comparePivot(int j) {
+        reader.getValue(j, scratch2);
+        int cmp = StringHelper.compare(numBytesToCompare, pivot, offset, scratch2, offset);
+        if (cmp == 0) {
+          cmp = pivotDoc - reader.getDocID(j);
+        }
+        return cmp;
+      }
+    }.sort(from, to);
+  }
+
+  /** Partition points around {@code mid}. All values on the left must be less
+   *  than or equal to it and all values on the right must be greater than or
+   *  equal to it. */
+  static void partition(int maxDoc, int splitDim, int bytesPerDim, int commonPrefixLen,
+      MutablePointsReader reader, int from, int to, int mid,
+      byte[] scratch1, byte[] scratch2) {
+    final int offset = splitDim * bytesPerDim + commonPrefixLen;
+    final int cmpBytes = bytesPerDim - commonPrefixLen;
+    final int bitsPerDocId = PackedInts.bitsRequired(maxDoc - 1);
+    new RadixSelector(cmpBytes + (bitsPerDocId + 7) / 8) {
+
+      @Override
+      protected Selector getFallbackSelector(int k) {
+        return new IntroSelector() {
+
+          final byte[] pivot = scratch1;
+          int pivotDoc;
+
+          @Override
+          protected void swap(int i, int j) {
+            reader.swap(i, j);
+          }
+
+          @Override
+          protected void setPivot(int i) {
+            reader.getValue(i, pivot);
+            pivotDoc = reader.getDocID(i);
+          }
+
+          @Override
+          protected int comparePivot(int j) {
+            if (k < cmpBytes) {
+              reader.getValue(j, scratch2);
+              int cmp = StringHelper.compare(cmpBytes - k, pivot, offset + k, scratch2, offset + k);
+              if (cmp != 0) {
+                return cmp;
+              }
+            }
+            return pivotDoc - reader.getDocID(j);
+          }
+        };
+      }
+
+      @Override
+      protected void swap(int i, int j) {
+        reader.swap(i, j);
+      }
+
+      @Override
+      protected int byteAt(int i, int k) {
+        if (k < cmpBytes) {
+          return Byte.toUnsignedInt(reader.getByteAt(i, offset + k));
+        } else {
+          final int shift = bitsPerDocId - ((k - cmpBytes + 1) << 3);
+          return (reader.getDocID(i) >>> Math.max(0, shift)) & 0xff;
+        }
+      }
+    }.select(from, to, mid);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/60975d2d/lucene/core/src/test/org/apache/lucene/util/TestByteBlockPool.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/TestByteBlockPool.java b/lucene/core/src/test/org/apache/lucene/util/TestByteBlockPool.java
index b425b76..388a789 100644
--- a/lucene/core/src/test/org/apache/lucene/util/TestByteBlockPool.java
+++ b/lucene/core/src/test/org/apache/lucene/util/TestByteBlockPool.java
@@ -45,7 +45,13 @@ public class TestByteBlockPool extends LuceneTestCase {
       for (BytesRef expected : list) {
         ref.grow(expected.length);
         ref.setLength(expected.length);
-        pool.readBytes(position, ref.bytes(), 0, ref.length());
+        if (random().nextBoolean()) {
+          pool.readBytes(position, ref.bytes(), 0, ref.length());
+        } else {
+          for (int i = 0; i < ref.length(); ++i) {
+            ref.setByteAt(i, pool.readByte(position + i));
+          }
+        }
         assertEquals(expected, ref.get());
         position += ref.length();
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/60975d2d/lucene/core/src/test/org/apache/lucene/util/TestIntroSelector.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/TestIntroSelector.java b/lucene/core/src/test/org/apache/lucene/util/TestIntroSelector.java
new file mode 100644
index 0000000..468b2f7
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/util/TestIntroSelector.java
@@ -0,0 +1,86 @@
+/*
+ * 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;
+
+import java.util.Arrays;
+
+public class TestIntroSelector extends LuceneTestCase {
+
+  public void testSelect() {
+    for (int iter = 0; iter < 100; ++iter) {
+      doTestSelect(false);
+    }
+  }
+
+  public void testSlowSelect() {
+    for (int iter = 0; iter < 100; ++iter) {
+      doTestSelect(true);
+    }
+  }
+
+  private void doTestSelect(boolean slow) {
+    final int from = random().nextInt(5);
+    final int to = from + TestUtil.nextInt(random(), 1, 10000);
+    final int max = random().nextBoolean() ? random().nextInt(100) : random().nextInt(100000);
+    Integer[] arr = new Integer[from + to + random().nextInt(5)];
+    for (int i = 0; i < arr.length; ++i) {
+      arr[i] = TestUtil.nextInt(random(), 0, max);
+    }
+    final int k = TestUtil.nextInt(random(), from, to - 1);
+
+    Integer[] expected = arr.clone();
+    Arrays.sort(expected, from, to);
+
+    Integer[] actual = arr.clone();
+    IntroSelector selector = new IntroSelector() {
+
+      Integer pivot;
+
+      @Override
+      protected void swap(int i, int j) {
+        ArrayUtil.swap(actual, i, j);
+      }
+
+      @Override
+      protected void setPivot(int i) {
+        pivot = actual[i];
+      }
+
+      @Override
+      protected int comparePivot(int j) {
+        return pivot.compareTo(actual[j]);
+      }
+    };
+    if (slow) {
+      selector.slowSelect(from, to, k);
+    } else {
+      selector.select(from, to, k);
+    }
+
+    assertEquals(expected[k], actual[k]);
+    for (int i = 0; i < actual.length; ++i) {
+      if (i < from || i >= to) {
+        assertSame(arr[i], actual[i]);
+      } else if (i <= k) {
+        assertTrue(actual[i].intValue() <= actual[k].intValue());
+      } else {
+        assertTrue(actual[i].intValue() >= actual[k].intValue());
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/60975d2d/lucene/core/src/test/org/apache/lucene/util/TestRadixSelector.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/TestRadixSelector.java b/lucene/core/src/test/org/apache/lucene/util/TestRadixSelector.java
new file mode 100644
index 0000000..e0d6bf9
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/util/TestRadixSelector.java
@@ -0,0 +1,77 @@
+/*
+ * 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;
+
+import java.util.Arrays;
+
+public class TestRadixSelector extends LuceneTestCase {
+
+  public void testSelect() {
+    for (int iter = 0; iter < 100; ++iter) {
+      doTestSelect();
+    }
+  }
+
+  private void doTestSelect() {
+    final int from = random().nextInt(5);
+    final int to = from + TestUtil.nextInt(random(), 1, 10000);
+    final int maxLen = TestUtil.nextInt(random(), 1, 12);
+    BytesRef[] arr = new BytesRef[from + to + random().nextInt(5)];
+    for (int i = 0; i < arr.length; ++i) {
+      byte[] bytes = new byte[TestUtil.nextInt(random(), 0, maxLen)];
+      random().nextBytes(bytes);
+      arr[i] = new BytesRef(bytes);
+    }
+    final int k = TestUtil.nextInt(random(), from, to - 1);
+
+    BytesRef[] expected = arr.clone();
+    Arrays.sort(expected, from, to);
+
+    BytesRef[] actual = arr.clone();
+    RadixSelector selector = new RadixSelector(random().nextBoolean() ? maxLen : Integer.MAX_VALUE) {
+
+      @Override
+      protected void swap(int i, int j) {
+        ArrayUtil.swap(actual, i, j);
+      }
+
+      @Override
+      protected int byteAt(int i, int k) {
+        BytesRef b = actual[i];
+        if (k >= b.length) {
+          return -1;
+        } else {
+          return Byte.toUnsignedInt(b.bytes[b.offset + k]);
+        }
+      }
+
+    };
+    selector.select(from, to, k);
+
+    assertEquals(expected[k], actual[k]);
+    for (int i = 0; i < actual.length; ++i) {
+      if (i < from || i >= to) {
+        assertSame(arr[i], actual[i]);
+      } else if (i <= k) {
+        assertTrue(actual[i].compareTo(actual[k]) <= 0);
+      } else {
+        assertTrue(actual[i].compareTo(actual[k]) >= 0);
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/60975d2d/lucene/core/src/test/org/apache/lucene/util/bkd/TestMutablePointsReaderUtils.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/bkd/TestMutablePointsReaderUtils.java b/lucene/core/src/test/org/apache/lucene/util/bkd/TestMutablePointsReaderUtils.java
new file mode 100644
index 0000000..f1140d4
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/util/bkd/TestMutablePointsReaderUtils.java
@@ -0,0 +1,251 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.bkd;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Comparator;
+
+import org.apache.lucene.codecs.MutablePointsReader;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.StringHelper;
+import org.apache.lucene.util.TestUtil;
+
+public class TestMutablePointsReaderUtils extends LuceneTestCase {
+
+  public void testSort() {
+    for (int iter = 0; iter < 5; ++iter) {
+      doTestSort();
+    }
+  }
+
+  private void doTestSort() {
+    final int bytesPerDim = TestUtil.nextInt(random(), 1, 16);
+    final int maxDoc = TestUtil.nextInt(random(), 1, 1 << random().nextInt(30));
+    Point[] points = createRandomPoints(1, bytesPerDim, maxDoc);
+    DummyPointsReader reader = new DummyPointsReader(points);
+    MutablePointsReaderUtils.sort(maxDoc, bytesPerDim, reader, 0, points.length);
+    Arrays.sort(points, new Comparator<Point>() {
+      @Override
+      public int compare(Point o1, Point o2) {
+        int cmp = StringHelper.compare(bytesPerDim, o1.packedValue, 0, o2.packedValue, 0);
+        if (cmp == 0) {
+          cmp = Integer.compare(o1.doc, o2.doc);
+        }
+        return cmp;
+      }
+    });
+    assertNotSame(points, reader.points);
+    assertArrayEquals(points, reader.points);
+  }
+
+  public void testSortByDim() {
+    for (int iter = 0; iter < 5; ++iter) {
+      doTestSortByDim();
+    }
+  }
+
+  private void doTestSortByDim() {
+    final int numDims = TestUtil.nextInt(random(), 1, 8);
+    final int bytesPerDim = TestUtil.nextInt(random(), 1, 16);
+    final int maxDoc = TestUtil.nextInt(random(), 1, 1 << random().nextInt(30));
+    Point[] points = createRandomPoints(numDims, bytesPerDim, maxDoc);
+    int[] commonPrefixLengths = new int[numDims];
+    for (int i = 0; i < commonPrefixLengths.length; ++i) {
+      commonPrefixLengths[i] = TestUtil.nextInt(random(), 0, bytesPerDim);
+    }
+    for (int i = 1; i < points.length; ++i) {
+      for (int dim = 0; dim < numDims; ++dim) {
+        int offset = dim * bytesPerDim;
+        System.arraycopy(points[0].packedValue, offset, points[i].packedValue, offset, commonPrefixLengths[dim]);
+      }
+    }
+    DummyPointsReader reader = new DummyPointsReader(points);
+    final int sortedDim = random().nextInt(numDims);
+    MutablePointsReaderUtils.sortByDim(sortedDim, bytesPerDim, commonPrefixLengths, reader, 0, points.length,
+        new byte[numDims * bytesPerDim], new byte[numDims * bytesPerDim]);
+    for (int i = 1; i < points.length; ++i) {
+      final int offset = sortedDim * bytesPerDim;
+      int cmp = StringHelper.compare(bytesPerDim, reader.points[i-1].packedValue, offset, reader.points[i].packedValue, offset);
+      if (cmp == 0) {
+        cmp = reader.points[i - 1].doc - reader.points[i].doc;
+      }
+      assertTrue(cmp <= 0);
+    }
+  }
+
+  public void testPartition() {
+    for (int iter = 0; iter < 5; ++iter) {
+      doTestPartition();
+    }
+  }
+
+  private void doTestPartition() {
+    final int numDims = TestUtil.nextInt(random(), 1, 8);
+    final int bytesPerDim = TestUtil.nextInt(random(), 1, 16);
+    final int maxDoc = TestUtil.nextInt(random(), 1, 1 << random().nextInt(30));
+    Point[] points = createRandomPoints(numDims, bytesPerDim, maxDoc);
+    int commonPrefixLength = TestUtil.nextInt(random(), 0, bytesPerDim);
+    final int splitDim =  random().nextInt(numDims);
+    for (int i = 1; i < points.length; ++i) {
+      int offset = splitDim * bytesPerDim;
+      System.arraycopy(points[0].packedValue, offset, points[i].packedValue, offset, commonPrefixLength);
+    }
+    DummyPointsReader reader = new DummyPointsReader(points);
+    final int pivot = TestUtil.nextInt(random(), 0, points.length - 1);
+    MutablePointsReaderUtils.partition(maxDoc, splitDim, bytesPerDim, commonPrefixLength, reader, 0, points.length, pivot,
+        new byte[numDims * bytesPerDim], new byte[numDims * bytesPerDim]);
+    int offset = splitDim * bytesPerDim;
+    for (int i = 0; i < points.length; ++i) {
+      int cmp = StringHelper.compare(bytesPerDim, reader.points[i].packedValue, offset, reader.points[pivot].packedValue, offset);
+      if (cmp == 0) {
+        cmp = reader.points[i].doc - reader.points[pivot].doc;
+      }
+      if (i < pivot) {
+        assertTrue(cmp <= 0);
+      } else if (i > pivot) {
+        assertTrue(cmp >= 0);
+      } else {
+        assertEquals(0, cmp);
+      }
+    }
+  }
+
+  private static Point[] createRandomPoints(int numDims, int bytesPerDim, int maxDoc) {
+    final int packedBytesLength = numDims * bytesPerDim;
+    final int numPoints = TestUtil.nextInt(random(), 1, 100000);
+    Point[] points = new Point[numPoints];
+    for (int i = 0; i < numPoints; ++i) {
+       byte[] value = new byte[packedBytesLength];
+       random().nextBytes(value);
+       points[i] = new Point(value, random().nextInt(maxDoc));
+    }
+    return points;
+  }
+
+  private static class Point {
+    final byte[] packedValue;
+    final int doc;
+
+    Point(byte[] packedValue, int doc) {
+      this.packedValue = packedValue;
+      this.doc = doc;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (obj == null || obj instanceof Point == false) {
+        return false;
+      }
+      Point that = (Point) obj;
+      return Arrays.equals(packedValue, that.packedValue) && doc == that.doc;
+    }
+
+    @Override
+    public int hashCode() {
+      return 31 * Arrays.hashCode(packedValue) + doc;
+    }
+
+    @Override
+    public String toString() {
+      return "value=" + new BytesRef(packedValue) + " doc=" + doc;
+    }
+  }
+
+  private static class DummyPointsReader extends MutablePointsReader {
+
+    private final Point[] points;
+
+    DummyPointsReader(Point[] points) {
+      this.points = points.clone();
+    }
+
+    @Override
+    public void close() throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long ramBytesUsed() {
+      return 0;
+    }
+
+    @Override
+    public void getValue(int i, byte[] packedValue) {
+      System.arraycopy(points[i].packedValue, 0, packedValue, 0, points[i].packedValue.length);
+    }
+
+    @Override
+    public byte getByteAt(int i, int k) {
+      return points[i].packedValue[k];
+    }
+
+    @Override
+    public int getDocID(int i) {
+      return points[i].doc;
+    }
+
+    @Override
+    public void swap(int i, int j) {
+      ArrayUtil.swap(points, i, j);
+    }
+
+    @Override
+    public void checkIntegrity() throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void intersect(String fieldName, IntersectVisitor visitor) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public byte[] getMinPackedValue(String fieldName) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public byte[] getMaxPackedValue(String fieldName) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int getNumDimensions(String fieldName) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int getBytesPerDimension(String fieldName) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long size(String fieldName) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int getDocCount(String fieldName) {
+      throw new UnsupportedOperationException();
+    }
+
+  }
+
+}


[23/40] lucene-solr:apiv2: Suppress fall-through warning in minmash's murmur

Posted by sa...@apache.org.
Suppress fall-through warning in minmash's murmur


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

Branch: refs/heads/apiv2
Commit: ef53ef9fcb4ba91dc583b44fc63fd32d94687841
Parents: 78ebcd3
Author: Uwe Schindler <us...@apache.org>
Authored: Thu Jul 28 11:21:34 2016 +0200
Committer: Uwe Schindler <us...@apache.org>
Committed: Thu Jul 28 11:21:34 2016 +0200

----------------------------------------------------------------------
 .../src/java/org/apache/lucene/analysis/minhash/MinHashFilter.java  | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ef53ef9f/lucene/analysis/common/src/java/org/apache/lucene/analysis/minhash/MinHashFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/minhash/MinHashFilter.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/minhash/MinHashFilter.java
index 1a1a637..61c816d 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/minhash/MinHashFilter.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/minhash/MinHashFilter.java
@@ -402,6 +402,7 @@ public class MinHashFilter extends TokenFilter {
   }
 
   /** Returns the MurmurHash3_x64_128 hash, placing the result in "out". */
+  @SuppressWarnings("fallthrough") // the huge switch is designed to use fall through into cases!
   static void murmurhash3_x64_128(byte[] key, int offset, int len, int seed, LongPair out) {
     // The original algorithm does have a 32 bit unsigned seed.
     // We have to mask to match the behavior of the unsigned types and prevent sign extension.


[02/40] lucene-solr:apiv2: SOLR-8995: Use lambdas in CoreAdminHandler

Posted by sa...@apache.org.
SOLR-8995: Use lambdas in CoreAdminHandler


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

Branch: refs/heads/apiv2
Commit: 941dff9bcda651de03f69671da12e39d9412ad35
Parents: d4cb52f
Author: Noble Paul <no...@apache.org>
Authored: Thu Jul 21 20:26:14 2016 +0530
Committer: Noble Paul <no...@apache.org>
Committed: Thu Jul 21 20:26:14 2016 +0530

----------------------------------------------------------------------
 .../solr/handler/admin/CoreAdminHandler.java    |    9 +-
 .../solr/handler/admin/CoreAdminOperation.java  | 1406 +++++++++---------
 2 files changed, 686 insertions(+), 729 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/941dff9b/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 49da174..7b0ecfb 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
@@ -362,7 +362,7 @@ public class CoreAdminHandler extends RequestHandlerBase implements PermissionNa
     }
 
     void call() throws Exception {
-      op.call(this);
+      op.execute(this);
     }
 
   }
@@ -374,7 +374,10 @@ public class CoreAdminHandler extends RequestHandlerBase implements PermissionNa
   /**
    * used by the INVOKE action of core admin handler
    */
-  public static interface Invocable {
-    public Map<String, Object> invoke(SolrQueryRequest req);
+  public interface Invocable {
+    Map<String, Object> invoke(SolrQueryRequest req);
+  }
+  interface CoreAdminOp {
+    void execute(CallInfo it) throws Exception;
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/941dff9b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java
index 33fe19a..9b9aafa 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java
@@ -64,6 +64,7 @@ import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.core.backup.repository.BackupRepository;
 import org.apache.solr.handler.RestoreCore;
 import org.apache.solr.handler.SnapShooter;
+import org.apache.solr.handler.admin.CoreAdminHandler.CoreAdminOp;
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.search.SolrIndexSearcher;
@@ -77,7 +78,6 @@ import org.apache.solr.util.NumberUtils;
 import org.apache.solr.util.PropertiesUtil;
 import org.apache.solr.util.RefCounted;
 import org.apache.solr.util.TestInjection;
-import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -96,839 +96,774 @@ import static org.apache.solr.handler.admin.CoreAdminHandler.RUNNING;
 import static org.apache.solr.handler.admin.CoreAdminHandler.buildCoreParams;
 import static org.apache.solr.handler.admin.CoreAdminHandler.normalizePath;
 
-enum CoreAdminOperation {
-  CREATE_OP(CREATE) {
-    @Override
-    public void call(CallInfo callInfo) {
-      assert TestInjection.injectRandomDelayInCoreCreation();
-      
-      SolrParams params = callInfo.req.getParams();
-      log.info("core create command {}", params);
-      String coreName = params.required().get(CoreAdminParams.NAME);
-      Map<String, String> coreParams = buildCoreParams(params);
-      CoreContainer coreContainer = callInfo.handler.coreContainer;
-      Path instancePath = coreContainer.getCoreRootDirectory().resolve(coreName);
-
-      // TODO: Should we nuke setting odd instance paths?  They break core discovery, generally
-      String instanceDir = callInfo.req.getParams().get(CoreAdminParams.INSTANCE_DIR);
-      if (instanceDir == null)
-        instanceDir = callInfo.req.getParams().get("property.instanceDir");
-      if (instanceDir != null) {
-        instanceDir = PropertiesUtil.substituteProperty(instanceDir, coreContainer.getContainerProperties());
-        instancePath = coreContainer.getCoreRootDirectory().resolve(instanceDir).normalize();
-      }
-
-      coreContainer.create(coreName, instancePath, coreParams);
-
-      callInfo.rsp.add("core", coreName);
+enum CoreAdminOperation implements CoreAdminOp {
+
+  CREATE_OP(CREATE, it -> {
+    assert TestInjection.injectRandomDelayInCoreCreation();
+
+    SolrParams params = it.req.getParams();
+    log().info("core create command {}", params);
+    String coreName = params.required().get(CoreAdminParams.NAME);
+    Map<String, String> coreParams = buildCoreParams(params);
+    CoreContainer coreContainer = it.handler.coreContainer;
+    Path instancePath = coreContainer.getCoreRootDirectory().resolve(coreName);
+
+    // TODO: Should we nuke setting odd instance paths?  They break core discovery, generally
+    String instanceDir = it.req.getParams().get(CoreAdminParams.INSTANCE_DIR);
+    if (instanceDir == null)
+      instanceDir = it.req.getParams().get("property.instanceDir");
+    if (instanceDir != null) {
+      instanceDir = PropertiesUtil.substituteProperty(instanceDir, coreContainer.getContainerProperties());
+      instancePath = coreContainer.getCoreRootDirectory().resolve(instanceDir).normalize();
     }
-  },
-  UNLOAD_OP(UNLOAD) {
-    @Override
-    public void call(CallInfo callInfo) throws IOException {
-      SolrParams params = callInfo.req.getParams();
-      String cname = params.get(CoreAdminParams.CORE);
-      boolean deleteIndexDir = params.getBool(CoreAdminParams.DELETE_INDEX, false);
-      boolean deleteDataDir = params.getBool(CoreAdminParams.DELETE_DATA_DIR, false);
-      boolean deleteInstanceDir = params.getBool(CoreAdminParams.DELETE_INSTANCE_DIR, false);
-      callInfo.handler.coreContainer.unload(cname, deleteIndexDir, deleteDataDir, deleteInstanceDir);
-
-      assert TestInjection.injectNonExistentCoreExceptionAfterUnload(cname);
-    }
-  },
-  RELOAD_OP(RELOAD) {
-    @Override
-    public void call(CallInfo callInfo) {
-      SolrParams params = callInfo.req.getParams();
-      String cname = params.get(CoreAdminParams.CORE);
 
-      if (cname == null || !callInfo.handler.coreContainer.getCoreNames().contains(cname)) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Core with core name [" + cname + "] does not exist.");
-      }
+    coreContainer.create(coreName, instancePath, coreParams);
 
-      try {
-        callInfo.handler.coreContainer.reload(cname);
-      } catch (Exception ex) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error handling 'reload' action", ex);
-      }
-    }
-  },
-  STATUS_OP(STATUS) {
-    @Override
-    public void call(CallInfo callInfo) {
-      SolrParams params = callInfo.req.getParams();
-
-      String cname = params.get(CoreAdminParams.CORE);
-      String indexInfo = params.get(CoreAdminParams.INDEX_INFO);
-      boolean isIndexInfoNeeded = Boolean.parseBoolean(null == indexInfo ? "true" : indexInfo);
-      NamedList<Object> status = new SimpleOrderedMap<>();
-      Map<String, Exception> failures = new HashMap<>();
-      for (Map.Entry<String, CoreContainer.CoreLoadFailure> failure : callInfo.handler.coreContainer.getCoreInitFailures().entrySet()) {
-        failures.put(failure.getKey(), failure.getValue().exception);
-      }
-      try {
-        if (cname == null) {
-          for (String name : callInfo.handler.coreContainer.getAllCoreNames()) {
-            status.add(name, getCoreStatus(callInfo.handler.coreContainer, name, isIndexInfoNeeded));
-          }
-          callInfo.rsp.add("initFailures", failures);
-        } else {
-          failures = failures.containsKey(cname)
-              ? Collections.singletonMap(cname, failures.get(cname))
-              : Collections.<String, Exception>emptyMap();
-          callInfo.rsp.add("initFailures", failures);
-          status.add(cname, getCoreStatus(callInfo.handler.coreContainer, cname, isIndexInfoNeeded));
-        }
-        callInfo.rsp.add("status", status);
-      } catch (Exception ex) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
-            "Error handling 'status' action ", ex);
-      }
-    }
+    it.rsp.add("core", coreName);
+  }),
+  UNLOAD_OP(UNLOAD, it -> {
+    SolrParams params = it.req.getParams();
+    String cname = params.get(CoreAdminParams.CORE);
+    boolean deleteIndexDir = params.getBool(CoreAdminParams.DELETE_INDEX, false);
+    boolean deleteDataDir = params.getBool(CoreAdminParams.DELETE_DATA_DIR, false);
+    boolean deleteInstanceDir = params.getBool(CoreAdminParams.DELETE_INSTANCE_DIR, false);
+    it.handler.coreContainer.unload(cname, deleteIndexDir, deleteDataDir, deleteInstanceDir);
 
+    assert TestInjection.injectNonExistentCoreExceptionAfterUnload(cname);
 
-  },
-
-  SWAP_OP(SWAP) {
-    @Override
-    public void call(CallInfo callInfo) {
-      final SolrParams params = callInfo.req.getParams();
-      final SolrParams required = params.required();
-
-      final String cname = params.get(CoreAdminParams.CORE);
-      String other = required.get(CoreAdminParams.OTHER);
-      callInfo.handler.coreContainer.swap(cname, other);
+  }),
+  RELOAD_OP(RELOAD, it -> {
+    SolrParams params = it.req.getParams();
+    String cname = params.get(CoreAdminParams.CORE);
 
+    if (cname == null || !it.handler.coreContainer.getCoreNames().contains(cname)) {
+      throw new SolrException(ErrorCode.BAD_REQUEST, "Core with core name [" + cname + "] does not exist.");
     }
-  },
-  RENAME_OP(RENAME) {
-    @Override
-    public void call(CallInfo callInfo) {
-      SolrParams params = callInfo.req.getParams();
 
-      String name = params.get(CoreAdminParams.OTHER);
-      String cname = params.get(CoreAdminParams.CORE);
-
-      if (cname.equals(name)) return;
-
-      callInfo.handler.coreContainer.rename(cname, name);
+    try {
+      it.handler.coreContainer.reload(cname);
+    } catch (Exception ex) {
+      throw new SolrException(ErrorCode.SERVER_ERROR, "Error handling 'reload' action", ex);
     }
-  },
-  MERGEINDEXES_OP(MERGEINDEXES) {
-    @Override
-    public void call(CallInfo callInfo) throws Exception {
-      SolrParams params = callInfo.req.getParams();
-      String cname = params.required().get(CoreAdminParams.CORE);
-      SolrCore core = callInfo.handler.coreContainer.getCore(cname);
-      SolrQueryRequest wrappedReq = null;
-
-      List<SolrCore> sourceCores = Lists.newArrayList();
-      List<RefCounted<SolrIndexSearcher>> searchers = Lists.newArrayList();
-      // stores readers created from indexDir param values
-      List<DirectoryReader> readersToBeClosed = Lists.newArrayList();
-      Map<Directory,Boolean> dirsToBeReleased = new HashMap<>();
-      if (core != null) {
-        try {
-          String[] dirNames = params.getParams(CoreAdminParams.INDEX_DIR);
-          if (dirNames == null || dirNames.length == 0) {
-            String[] sources = params.getParams("srcCore");
-            if (sources == null || sources.length == 0)
-              throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-                  "At least one indexDir or srcCore must be specified");
-
-            for (int i = 0; i < sources.length; i++) {
-              String source = sources[i];
-              SolrCore srcCore = callInfo.handler.coreContainer.getCore(source);
-              if (srcCore == null)
-                throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-                    "Core: " + source + " does not exist");
-              sourceCores.add(srcCore);
-            }
-          } else {
-            DirectoryFactory dirFactory = core.getDirectoryFactory();
-            for (int i = 0; i < dirNames.length; i++) {
-              boolean markAsDone = false;
-              if (dirFactory instanceof CachingDirectoryFactory) {
-                if (!((CachingDirectoryFactory)dirFactory).getLivePaths().contains(dirNames[i])) {
-                  markAsDone = true;
-                }
-              }
-              Directory dir = dirFactory.get(dirNames[i], DirectoryFactory.DirContext.DEFAULT, core.getSolrConfig().indexConfig.lockType);
-              dirsToBeReleased.put(dir, markAsDone);
-              // TODO: why doesn't this use the IR factory? what is going on here?
-              readersToBeClosed.add(DirectoryReader.open(dir));
-            }
+  }),
+
+  STATUS_OP(STATUS, it -> {
+    SolrParams params = it.req.getParams();
+
+    String cname = params.get(CoreAdminParams.CORE);
+    String indexInfo = params.get(CoreAdminParams.INDEX_INFO);
+    boolean isIndexInfoNeeded = Boolean.parseBoolean(null == indexInfo ? "true" : indexInfo);
+    NamedList<Object> status = new SimpleOrderedMap<>();
+    Map<String, Exception> failures = new HashMap<>();
+    for (Entry<String, CoreContainer.CoreLoadFailure> failure : it.handler.coreContainer.getCoreInitFailures().entrySet()) {
+      failures.put(failure.getKey(), failure.getValue().exception);
+    }
+    try {
+      if (cname == null) {
+        for (String name : it.handler.coreContainer.getAllCoreNames()) {
+          status.add(name, getCoreStatus(it.handler.coreContainer, name, isIndexInfoNeeded));
+        }
+        it.rsp.add("initFailures", failures);
+      } else {
+        failures = failures.containsKey(cname)
+            ? Collections.singletonMap(cname, failures.get(cname))
+            : Collections.<String, Exception>emptyMap();
+        it.rsp.add("initFailures", failures);
+        status.add(cname, getCoreStatus(it.handler.coreContainer, cname, isIndexInfoNeeded));
+      }
+      it.rsp.add("status", status);
+    } catch (Exception ex) {
+      throw new SolrException(ErrorCode.SERVER_ERROR,
+          "Error handling 'status' action ", ex);
+    }
+  }),
+
+  SWAP_OP(SWAP, it -> {
+    final SolrParams params = it.req.getParams();
+    final String cname = params.get(CoreAdminParams.CORE);
+    String other = params.required().get(CoreAdminParams.OTHER);
+    it.handler.coreContainer.swap(cname, other);
+  }),
+
+  RENAME_OP(RENAME, it -> {
+    SolrParams params = it.req.getParams();
+    String name = params.get(CoreAdminParams.OTHER);
+    String cname = params.get(CoreAdminParams.CORE);
+
+    if (cname.equals(name)) return;
+
+    it.handler.coreContainer.rename(cname, name);
+  }),
+
+  MERGEINDEXES_OP(MERGEINDEXES, it -> {
+    SolrParams params = it.req.getParams();
+    String cname = params.required().get(CoreAdminParams.CORE);
+    SolrCore core = it.handler.coreContainer.getCore(cname);
+    SolrQueryRequest wrappedReq = null;
+
+    List<SolrCore> sourceCores = Lists.newArrayList();
+    List<RefCounted<SolrIndexSearcher>> searchers = Lists.newArrayList();
+    // stores readers created from indexDir param values
+    List<DirectoryReader> readersToBeClosed = Lists.newArrayList();
+    Map<Directory, Boolean> dirsToBeReleased = new HashMap<>();
+    if (core != null) {
+      try {
+        String[] dirNames = params.getParams(CoreAdminParams.INDEX_DIR);
+        if (dirNames == null || dirNames.length == 0) {
+          String[] sources = params.getParams("srcCore");
+          if (sources == null || sources.length == 0)
+            throw new SolrException(ErrorCode.BAD_REQUEST,
+                "At least one indexDir or srcCore must be specified");
+
+          for (int i = 0; i < sources.length; i++) {
+            String source = sources[i];
+            SolrCore srcCore = it.handler.coreContainer.getCore(source);
+            if (srcCore == null)
+              throw new SolrException(ErrorCode.BAD_REQUEST,
+                  "Core: " + source + " does not exist");
+            sourceCores.add(srcCore);
           }
-
-          List<DirectoryReader> readers = null;
-          if (readersToBeClosed.size() > 0) {
-            readers = readersToBeClosed;
-          } else {
-            readers = Lists.newArrayList();
-            for (SolrCore solrCore : sourceCores) {
-              // record the searchers so that we can decref
-              RefCounted<SolrIndexSearcher> searcher = solrCore.getSearcher();
-              searchers.add(searcher);
-              readers.add(searcher.get().getIndexReader());
+        } else {
+          DirectoryFactory dirFactory = core.getDirectoryFactory();
+          for (int i = 0; i < dirNames.length; i++) {
+            boolean markAsDone = false;
+            if (dirFactory instanceof CachingDirectoryFactory) {
+              if (!((CachingDirectoryFactory) dirFactory).getLivePaths().contains(dirNames[i])) {
+                markAsDone = true;
+              }
             }
+            Directory dir = dirFactory.get(dirNames[i], DirectoryFactory.DirContext.DEFAULT, core.getSolrConfig().indexConfig.lockType);
+            dirsToBeReleased.put(dir, markAsDone);
+            // TODO: why doesn't this use the IR factory? what is going on here?
+            readersToBeClosed.add(DirectoryReader.open(dir));
           }
+        }
 
-          UpdateRequestProcessorChain processorChain =
-              core.getUpdateProcessingChain(params.get(UpdateParams.UPDATE_CHAIN));
-          wrappedReq = new LocalSolrQueryRequest(core, callInfo.req.getParams());
-          UpdateRequestProcessor processor =
-              processorChain.createProcessor(wrappedReq, callInfo.rsp);
-          processor.processMergeIndexes(new MergeIndexesCommand(readers, callInfo.req));
-        } catch (Exception e) {
-          // log and rethrow so that if the finally fails we don't lose the original problem
-          log.error("ERROR executing merge:", e);
-          throw e;
-        } finally {
-          for (RefCounted<SolrIndexSearcher> searcher : searchers) {
-            if (searcher != null) searcher.decref();
-          }
+        List<DirectoryReader> readers = null;
+        if (readersToBeClosed.size() > 0) {
+          readers = readersToBeClosed;
+        } else {
+          readers = Lists.newArrayList();
           for (SolrCore solrCore : sourceCores) {
-            if (solrCore != null) solrCore.close();
+            // record the searchers so that we can decref
+            RefCounted<SolrIndexSearcher> searcher = solrCore.getSearcher();
+            searchers.add(searcher);
+            readers.add(searcher.get().getIndexReader());
           }
-          IOUtils.closeWhileHandlingException(readersToBeClosed);
-          Set<Entry<Directory,Boolean>> entries = dirsToBeReleased.entrySet();
-          for (Entry<Directory,Boolean> entry : entries) {
-            DirectoryFactory dirFactory = core.getDirectoryFactory();
-            Directory dir = entry.getKey();
-            boolean markAsDone = entry.getValue();
-            if (markAsDone) {
-              dirFactory.doneWithDirectory(dir);
-            }
-            dirFactory.release(dir);
+        }
+
+        UpdateRequestProcessorChain processorChain =
+            core.getUpdateProcessingChain(params.get(UpdateParams.UPDATE_CHAIN));
+        wrappedReq = new LocalSolrQueryRequest(core, it.req.getParams());
+        UpdateRequestProcessor processor =
+            processorChain.createProcessor(wrappedReq, it.rsp);
+        processor.processMergeIndexes(new MergeIndexesCommand(readers, it.req));
+      } catch (Exception e) {
+        // log and rethrow so that if the finally fails we don't lose the original problem
+        log().error("ERROR executing merge:", e);
+        throw e;
+      } finally {
+        for (RefCounted<SolrIndexSearcher> searcher : searchers) {
+          if (searcher != null) searcher.decref();
+        }
+        for (SolrCore solrCore : sourceCores) {
+          if (solrCore != null) solrCore.close();
+        }
+        IOUtils.closeWhileHandlingException(readersToBeClosed);
+        Set<Entry<Directory, Boolean>> entries = dirsToBeReleased.entrySet();
+        for (Entry<Directory, Boolean> entry : entries) {
+          DirectoryFactory dirFactory = core.getDirectoryFactory();
+          Directory dir = entry.getKey();
+          boolean markAsDone = entry.getValue();
+          if (markAsDone) {
+            dirFactory.doneWithDirectory(dir);
           }
-          if (wrappedReq != null) wrappedReq.close();
-          core.close();
+          dirFactory.release(dir);
         }
+        if (wrappedReq != null) wrappedReq.close();
+        core.close();
       }
-
     }
-  },
-  SPLIT_OP(SPLIT) {
-    @Override
-    public void call(CallInfo callInfo) throws IOException {
-      SolrParams params = callInfo.req.getParams();
-      List<DocRouter.Range> ranges = null;
-
-      String[] pathsArr = params.getParams(PATH);
-      String rangesStr = params.get(CoreAdminParams.RANGES);    // ranges=a-b,c-d,e-f
-      if (rangesStr != null) {
-        String[] rangesArr = rangesStr.split(",");
-        if (rangesArr.length == 0) {
-          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "There must be at least one range specified to split an index");
-        } else {
-          ranges = new ArrayList<>(rangesArr.length);
-          for (String r : rangesArr) {
-            try {
-              ranges.add(DocRouter.DEFAULT.fromString(r));
-            } catch (Exception e) {
-              throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Exception parsing hexadecimal hash range: " + r, e);
-            }
+  }),
+
+  SPLIT_OP(SPLIT, it -> {
+    SolrParams params = it.req.getParams();
+    List<DocRouter.Range> ranges = null;
+
+    String[] pathsArr = params.getParams(PATH);
+    String rangesStr = params.get(CoreAdminParams.RANGES);    // ranges=a-b,c-d,e-f
+    if (rangesStr != null) {
+      String[] rangesArr = rangesStr.split(",");
+      if (rangesArr.length == 0) {
+        throw new SolrException(ErrorCode.BAD_REQUEST, "There must be at least one range specified to split an index");
+      } else {
+        ranges = new ArrayList<>(rangesArr.length);
+        for (String r : rangesArr) {
+          try {
+            ranges.add(DocRouter.DEFAULT.fromString(r));
+          } catch (Exception e) {
+            throw new SolrException(ErrorCode.BAD_REQUEST, "Exception parsing hexadecimal hash range: " + r, e);
           }
         }
       }
-      String splitKey = params.get("split.key");
-      String[] newCoreNames = params.getParams("targetCore");
-      String cname = params.get(CoreAdminParams.CORE, "");
+    }
+    String splitKey = params.get("split.key");
+    String[] newCoreNames = params.getParams("targetCore");
+    String cname = params.get(CoreAdminParams.CORE, "");
 
-      if ((pathsArr == null || pathsArr.length == 0) && (newCoreNames == null || newCoreNames.length == 0)) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Either path or targetCore param must be specified");
-      }
+    if ((pathsArr == null || pathsArr.length == 0) && (newCoreNames == null || newCoreNames.length == 0)) {
+      throw new SolrException(ErrorCode.BAD_REQUEST, "Either path or targetCore param must be specified");
+    }
 
-      log.info("Invoked split action for core: " + cname);
-      SolrCore core = callInfo.handler.coreContainer.getCore(cname);
-      SolrQueryRequest req = new LocalSolrQueryRequest(core, params);
-      List<SolrCore> newCores = null;
+    log().info("Invoked split action for core: " + cname);
+    SolrCore core = it.handler.coreContainer.getCore(cname);
+    SolrQueryRequest req = new LocalSolrQueryRequest(core, params);
+    List<SolrCore> newCores = null;
 
-      try {
-        // TODO: allow use of rangesStr in the future
-        List<String> paths = null;
-        int partitions = pathsArr != null ? pathsArr.length : newCoreNames.length;
-
-        DocRouter router = null;
-        String routeFieldName = null;
-        if (callInfo.handler.coreContainer.isZooKeeperAware()) {
-          ClusterState clusterState = callInfo.handler.coreContainer.getZkController().getClusterState();
-          String collectionName = req.getCore().getCoreDescriptor().getCloudDescriptor().getCollectionName();
-          DocCollection collection = clusterState.getCollection(collectionName);
-          String sliceName = req.getCore().getCoreDescriptor().getCloudDescriptor().getShardId();
-          Slice slice = collection.getSlice(sliceName);
-          router = collection.getRouter() != null ? collection.getRouter() : DocRouter.DEFAULT;
-          if (ranges == null) {
-            DocRouter.Range currentRange = slice.getRange();
-            ranges = currentRange != null ? router.partitionRange(partitions, currentRange) : null;
-          }
-          Object routerObj = collection.get(DOC_ROUTER); // for back-compat with Solr 4.4
-          if (routerObj != null && routerObj instanceof Map) {
-            Map routerProps = (Map) routerObj;
-            routeFieldName = (String) routerProps.get("field");
-          }
+    try {
+      // TODO: allow use of rangesStr in the future
+      List<String> paths = null;
+      int partitions = pathsArr != null ? pathsArr.length : newCoreNames.length;
+
+      DocRouter router = null;
+      String routeFieldName = null;
+      if (it.handler.coreContainer.isZooKeeperAware()) {
+        ClusterState clusterState = it.handler.coreContainer.getZkController().getClusterState();
+        String collectionName = req.getCore().getCoreDescriptor().getCloudDescriptor().getCollectionName();
+        DocCollection collection = clusterState.getCollection(collectionName);
+        String sliceName = req.getCore().getCoreDescriptor().getCloudDescriptor().getShardId();
+        Slice slice = collection.getSlice(sliceName);
+        router = collection.getRouter() != null ? collection.getRouter() : DocRouter.DEFAULT;
+        if (ranges == null) {
+          DocRouter.Range currentRange = slice.getRange();
+          ranges = currentRange != null ? router.partitionRange(partitions, currentRange) : null;
         }
+        Object routerObj = collection.get(DOC_ROUTER); // for back-compat with Solr 4.4
+        if (routerObj != null && routerObj instanceof Map) {
+          Map routerProps = (Map) routerObj;
+          routeFieldName = (String) routerProps.get("field");
+        }
+      }
 
-        if (pathsArr == null) {
-          newCores = new ArrayList<>(partitions);
-          for (String newCoreName : newCoreNames) {
-            SolrCore newcore = callInfo.handler.coreContainer.getCore(newCoreName);
-            if (newcore != null) {
-              newCores.add(newcore);
-            } else {
-              throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Core with core name " + newCoreName + " expected but doesn't exist.");
-            }
+      if (pathsArr == null) {
+        newCores = new ArrayList<>(partitions);
+        for (String newCoreName : newCoreNames) {
+          SolrCore newcore = it.handler.coreContainer.getCore(newCoreName);
+          if (newcore != null) {
+            newCores.add(newcore);
+          } else {
+            throw new SolrException(ErrorCode.BAD_REQUEST, "Core with core name " + newCoreName + " expected but doesn't exist.");
           }
-        } else {
-          paths = Arrays.asList(pathsArr);
         }
+      } else {
+        paths = Arrays.asList(pathsArr);
+      }
 
 
-        SplitIndexCommand cmd = new SplitIndexCommand(req, paths, newCores, ranges, router, routeFieldName, splitKey);
-        core.getUpdateHandler().split(cmd);
+      SplitIndexCommand cmd = new SplitIndexCommand(req, paths, newCores, ranges, router, routeFieldName, splitKey);
+      core.getUpdateHandler().split(cmd);
 
-        // After the split has completed, someone (here?) should start the process of replaying the buffered updates.
+      // After the split has completed, someone (here?) should start the process of replaying the buffered updates.
 
-      } catch (Exception e) {
-        log.error("ERROR executing split:", e);
-        throw new RuntimeException(e);
+    } catch (Exception e) {
+      log().error("ERROR executing split:", e);
+      throw new RuntimeException(e);
 
-      } finally {
-        if (req != null) req.close();
-        if (core != null) core.close();
-        if (newCores != null) {
-          for (SolrCore newCore : newCores) {
-            newCore.close();
-          }
+    } finally {
+      if (req != null) req.close();
+      if (core != null) core.close();
+      if (newCores != null) {
+        for (SolrCore newCore : newCores) {
+          newCore.close();
         }
       }
-
     }
-  },
-  PREPRECOVERY_OP(PREPRECOVERY) {
-    @Override
-    public void call(CallInfo callInfo) throws InterruptedException, IOException, KeeperException {
-      final SolrParams params = callInfo.req.getParams();
+  }),
 
-      String cname = params.get(CoreAdminParams.CORE);
-      if (cname == null) {
-        cname = "";
-      }
+  PREPRECOVERY_OP(PREPRECOVERY, it -> {
+    final SolrParams params = it.req.getParams();
 
-      String nodeName = params.get("nodeName");
-      String coreNodeName = params.get("coreNodeName");
-      Replica.State waitForState = Replica.State.getState(params.get(ZkStateReader.STATE_PROP));
-      Boolean checkLive = params.getBool("checkLive");
-      Boolean onlyIfLeader = params.getBool("onlyIfLeader");
-      Boolean onlyIfLeaderActive = params.getBool("onlyIfLeaderActive");
-
-      log.info("Going to wait for coreNodeName: " + coreNodeName + ", state: " + waitForState
-          + ", checkLive: " + checkLive + ", onlyIfLeader: " + onlyIfLeader
-          + ", onlyIfLeaderActive: " + onlyIfLeaderActive);
-
-      int maxTries = 0;
-      Replica.State state = null;
-      boolean live = false;
-      int retry = 0;
-      while (true) {
-        CoreContainer coreContainer = callInfo.handler.coreContainer;
-        try (SolrCore core = coreContainer.getCore(cname)) {
-          if (core == null && retry == 30) {
-            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "core not found:"
-                + cname);
-          }
-          if (core != null) {
-            if (onlyIfLeader != null && onlyIfLeader) {
-              if (!core.getCoreDescriptor().getCloudDescriptor().isLeader()) {
-                throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "We are not the leader");
-              }
-            }
+    String cname = params.get(CoreAdminParams.CORE);
+    if (cname == null) {
+      cname = "";
+    }
 
-            // wait until we are sure the recovering node is ready
-            // to accept updates
-            CloudDescriptor cloudDescriptor = core.getCoreDescriptor()
-                .getCloudDescriptor();
-            String collectionName = cloudDescriptor.getCollectionName();
-
-            if (retry % 15 == 0) {
-              if (retry > 0 && log.isInfoEnabled())
-                log.info("After " + retry + " seconds, core " + cname + " (" +
-                    cloudDescriptor.getShardId() + " of " +
-                    cloudDescriptor.getCollectionName() + ") still does not have state: " +
-                    waitForState + "; forcing ClusterState update from ZooKeeper");
-
-              // force a cluster state update
-              coreContainer.getZkController().getZkStateReader().forceUpdateCollection(collectionName);
+    String nodeName = params.get("nodeName");
+    String coreNodeName = params.get("coreNodeName");
+    Replica.State waitForState = Replica.State.getState(params.get(ZkStateReader.STATE_PROP));
+    Boolean checkLive = params.getBool("checkLive");
+    Boolean onlyIfLeader = params.getBool("onlyIfLeader");
+    Boolean onlyIfLeaderActive = params.getBool("onlyIfLeaderActive");
+
+    log().info("Going to wait for coreNodeName: " + coreNodeName + ", state: " + waitForState
+        + ", checkLive: " + checkLive + ", onlyIfLeader: " + onlyIfLeader
+        + ", onlyIfLeaderActive: " + onlyIfLeaderActive);
+
+    int maxTries = 0;
+    Replica.State state = null;
+    boolean live = false;
+    int retry = 0;
+    while (true) {
+      CoreContainer coreContainer = it.handler.coreContainer;
+      try (SolrCore core = coreContainer.getCore(cname)) {
+        if (core == null && retry == 30) {
+          throw new SolrException(ErrorCode.BAD_REQUEST, "core not found:"
+              + cname);
+        }
+        if (core != null) {
+          if (onlyIfLeader != null && onlyIfLeader) {
+            if (!core.getCoreDescriptor().getCloudDescriptor().isLeader()) {
+              throw new SolrException(ErrorCode.BAD_REQUEST, "We are not the leader");
             }
+          }
+
+          // wait until we are sure the recovering node is ready
+          // to accept updates
+          CloudDescriptor cloudDescriptor = core.getCoreDescriptor()
+              .getCloudDescriptor();
+          String collectionName = cloudDescriptor.getCollectionName();
 
-            if (maxTries == 0) {
-              // wait long enough for the leader conflict to work itself out plus a little extra
-              int conflictWaitMs = coreContainer.getZkController().getLeaderConflictResolveWait();
-              maxTries = (int) Math.round(conflictWaitMs / 1000) + 3;
-              log.info("Will wait a max of " + maxTries + " seconds to see " + cname + " (" +
+          if (retry % 15 == 0) {
+            if (retry > 0 && log().isInfoEnabled())
+              log().info("After " + retry + " seconds, core " + cname + " (" +
                   cloudDescriptor.getShardId() + " of " +
-                  cloudDescriptor.getCollectionName() + ") have state: " + waitForState);
-            }
+                  cloudDescriptor.getCollectionName() + ") still does not have state: " +
+                  waitForState + "; forcing ClusterState update from ZooKeeper");
 
-            ClusterState clusterState = coreContainer.getZkController().getClusterState();
-            DocCollection collection = clusterState.getCollection(collectionName);
-            Slice slice = collection.getSlice(cloudDescriptor.getShardId());
-            if (slice != null) {
-              final Replica replica = slice.getReplicasMap().get(coreNodeName);
-              if (replica != null) {
-                state = replica.getState();
-                live = clusterState.liveNodesContain(nodeName);
-
-                final Replica.State localState = cloudDescriptor.getLastPublished();
-
-                // TODO: This is funky but I've seen this in testing where the replica asks the
-                // leader to be in recovery? Need to track down how that happens ... in the meantime,
-                // this is a safeguard
-                boolean leaderDoesNotNeedRecovery = (onlyIfLeader != null &&
-                    onlyIfLeader &&
-                    core.getName().equals(replica.getStr("core")) &&
-                    waitForState == Replica.State.RECOVERING &&
-                    localState == Replica.State.ACTIVE &&
-                    state == Replica.State.ACTIVE);
-
-                if (leaderDoesNotNeedRecovery) {
-                  log.warn("Leader " + core.getName() + " ignoring request to be in the recovering state because it is live and active.");
-                }
+            // force a cluster state update
+            coreContainer.getZkController().getZkStateReader().forceUpdateCollection(collectionName);
+          }
 
-                boolean onlyIfActiveCheckResult = onlyIfLeaderActive != null && onlyIfLeaderActive && localState != Replica.State.ACTIVE;
-                log.info("In WaitForState(" + waitForState + "): collection=" + collectionName + ", shard=" + slice.getName() +
-                    ", thisCore=" + core.getName() + ", leaderDoesNotNeedRecovery=" + leaderDoesNotNeedRecovery +
-                    ", isLeader? " + core.getCoreDescriptor().getCloudDescriptor().isLeader() +
-                    ", live=" + live + ", checkLive=" + checkLive + ", currentState=" + state.toString() + ", localState=" + localState + ", nodeName=" + nodeName +
-                    ", coreNodeName=" + coreNodeName + ", onlyIfActiveCheckResult=" + onlyIfActiveCheckResult + ", nodeProps: " + replica);
-
-                if (!onlyIfActiveCheckResult && replica != null && (state == waitForState || leaderDoesNotNeedRecovery)) {
-                  if (checkLive == null) {
-                    break;
-                  } else if (checkLive && live) {
-                    break;
-                  } else if (!checkLive && !live) {
-                    break;
-                  }
+          if (maxTries == 0) {
+            // wait long enough for the leader conflict to work itself out plus a little extra
+            int conflictWaitMs = coreContainer.getZkController().getLeaderConflictResolveWait();
+            maxTries = (int) Math.round(conflictWaitMs / 1000) + 3;
+            log().info("Will wait a max of " + maxTries + " seconds to see " + cname + " (" +
+                cloudDescriptor.getShardId() + " of " +
+                cloudDescriptor.getCollectionName() + ") have state: " + waitForState);
+          }
+
+          ClusterState clusterState = coreContainer.getZkController().getClusterState();
+          DocCollection collection = clusterState.getCollection(collectionName);
+          Slice slice = collection.getSlice(cloudDescriptor.getShardId());
+          if (slice != null) {
+            final Replica replica = slice.getReplicasMap().get(coreNodeName);
+            if (replica != null) {
+              state = replica.getState();
+              live = clusterState.liveNodesContain(nodeName);
+
+              final Replica.State localState = cloudDescriptor.getLastPublished();
+
+              // TODO: This is funky but I've seen this in testing where the replica asks the
+              // leader to be in recovery? Need to track down how that happens ... in the meantime,
+              // this is a safeguard
+              boolean leaderDoesNotNeedRecovery = (onlyIfLeader != null &&
+                  onlyIfLeader &&
+                  core.getName().equals(replica.getStr("core")) &&
+                  waitForState == Replica.State.RECOVERING &&
+                  localState == Replica.State.ACTIVE &&
+                  state == Replica.State.ACTIVE);
+
+              if (leaderDoesNotNeedRecovery) {
+                log().warn("Leader " + core.getName() + " ignoring request to be in the recovering state because it is live and active.");
+              }
+
+              boolean onlyIfActiveCheckResult = onlyIfLeaderActive != null && onlyIfLeaderActive && localState != Replica.State.ACTIVE;
+              log().info("In WaitForState(" + waitForState + "): collection=" + collectionName + ", shard=" + slice.getName() +
+                  ", thisCore=" + core.getName() + ", leaderDoesNotNeedRecovery=" + leaderDoesNotNeedRecovery +
+                  ", isLeader? " + core.getCoreDescriptor().getCloudDescriptor().isLeader() +
+                  ", live=" + live + ", checkLive=" + checkLive + ", currentState=" + state.toString() + ", localState=" + localState + ", nodeName=" + nodeName +
+                  ", coreNodeName=" + coreNodeName + ", onlyIfActiveCheckResult=" + onlyIfActiveCheckResult + ", nodeProps: " + replica);
+
+              if (!onlyIfActiveCheckResult && replica != null && (state == waitForState || leaderDoesNotNeedRecovery)) {
+                if (checkLive == null) {
+                  break;
+                } else if (checkLive && live) {
+                  break;
+                } else if (!checkLive && !live) {
+                  break;
                 }
               }
             }
           }
+        }
 
-          if (retry++ == maxTries) {
-            String collection = null;
-            String leaderInfo = null;
-            String shardId = null;
-            try {
-              CloudDescriptor cloudDescriptor =
-                  core.getCoreDescriptor().getCloudDescriptor();
-              collection = cloudDescriptor.getCollectionName();
-              shardId = cloudDescriptor.getShardId();
-              leaderInfo = coreContainer.getZkController().
-                  getZkStateReader().getLeaderUrl(collection, shardId, 5000);
-            } catch (Exception exc) {
-              leaderInfo = "Not available due to: " + exc;
-            }
-
-            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-                "I was asked to wait on state " + waitForState + " for "
-                    + shardId + " in " + collection + " on " + nodeName
-                    + " but I still do not see the requested state. I see state: "
-                    + state.toString() + " live:" + live + " leader from ZK: " + leaderInfo
-            );
+        if (retry++ == maxTries) {
+          String collection = null;
+          String leaderInfo = null;
+          String shardId = null;
+          try {
+            CloudDescriptor cloudDescriptor =
+                core.getCoreDescriptor().getCloudDescriptor();
+            collection = cloudDescriptor.getCollectionName();
+            shardId = cloudDescriptor.getShardId();
+            leaderInfo = coreContainer.getZkController().
+                getZkStateReader().getLeaderUrl(collection, shardId, 5000);
+          } catch (Exception exc) {
+            leaderInfo = "Not available due to: " + exc;
           }
 
-          if (coreContainer.isShutDown()) {
-            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-                "Solr is shutting down");
-          }
+          throw new SolrException(ErrorCode.BAD_REQUEST,
+              "I was asked to wait on state " + waitForState + " for "
+                  + shardId + " in " + collection + " on " + nodeName
+                  + " but I still do not see the requested state. I see state: "
+                  + state.toString() + " live:" + live + " leader from ZK: " + leaderInfo
+          );
+        }
+
+        if (coreContainer.isShutDown()) {
+          throw new SolrException(ErrorCode.BAD_REQUEST,
+              "Solr is shutting down");
+        }
 
-          // solrcloud_debug
-          if (log.isDebugEnabled()) {
+        // solrcloud_debug
+        if (log().isDebugEnabled()) {
+          try {
+            LocalSolrQueryRequest r = new LocalSolrQueryRequest(core,
+                new ModifiableSolrParams());
+            CommitUpdateCommand commitCmd = new CommitUpdateCommand(r, false);
+            commitCmd.softCommit = true;
+            core.getUpdateHandler().commit(commitCmd);
+            RefCounted<SolrIndexSearcher> searchHolder = core
+                .getNewestSearcher(false);
+            SolrIndexSearcher searcher = searchHolder.get();
             try {
-              LocalSolrQueryRequest r = new LocalSolrQueryRequest(core,
-                  new ModifiableSolrParams());
-              CommitUpdateCommand commitCmd = new CommitUpdateCommand(r, false);
-              commitCmd.softCommit = true;
-              core.getUpdateHandler().commit(commitCmd);
-              RefCounted<SolrIndexSearcher> searchHolder = core
-                  .getNewestSearcher(false);
-              SolrIndexSearcher searcher = searchHolder.get();
-              try {
-                log.debug(core.getCoreDescriptor().getCoreContainer()
-                    .getZkController().getNodeName()
-                    + " to replicate "
-                    + searcher.search(new MatchAllDocsQuery(), 1).totalHits
-                    + " gen:"
-                    + core.getDeletionPolicy().getLatestCommit().getGeneration()
-                    + " data:" + core.getDataDir());
-              } finally {
-                searchHolder.decref();
-              }
-            } catch (Exception e) {
-              log.debug("Error in solrcloud_debug block", e);
+              log().debug(core.getCoreDescriptor().getCoreContainer()
+                  .getZkController().getNodeName()
+                  + " to replicate "
+                  + searcher.search(new MatchAllDocsQuery(), 1).totalHits
+                  + " gen:"
+                  + core.getDeletionPolicy().getLatestCommit().getGeneration()
+                  + " data:" + core.getDataDir());
+            } finally {
+              searchHolder.decref();
             }
+          } catch (Exception e) {
+            log().debug("Error in solrcloud_debug block", e);
           }
         }
-        Thread.sleep(1000);
       }
-
-      log.info("Waited coreNodeName: " + coreNodeName + ", state: " + waitForState
-          + ", checkLive: " + checkLive + ", onlyIfLeader: " + onlyIfLeader + " for: " + retry + " seconds.");
-    }
-  },
-  REQUESTRECOVERY_OP(REQUESTRECOVERY) {
-    @Override
-    public void call(final CallInfo callInfo) throws IOException {
-      final SolrParams params = callInfo.req.getParams();
-      log.info("It has been requested that we recover: core="+params.get(CoreAdminParams.CORE));
-      new Thread(() -> {
-        String cname = params.get(CoreAdminParams.CORE);
-        if (cname == null) {
-          cname = "";
-        }
-        try (SolrCore core = callInfo.handler.coreContainer.getCore(cname)) {
-          if (core != null) {
-            core.getUpdateHandler().getSolrCoreState().doRecovery(callInfo.handler.coreContainer, core.getCoreDescriptor());
-          } else {
-            SolrException.log(log, "Could not find core to call recovery:" + cname);
-          }
-        }
-      }).start();
-
+      Thread.sleep(1000);
     }
-  },
-  REQUESTSYNCSHARD_OP(REQUESTSYNCSHARD) {
-    @Override
-    public void call(CallInfo callInfo) throws IOException {
-      final SolrParams params = callInfo.req.getParams();
 
-      log.info("I have been requested to sync up my shard");
-      ZkController zkController = callInfo.handler.coreContainer.getZkController();
-      if (zkController == null) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Only valid for SolrCloud");
-      }
+    log().info("Waited coreNodeName: " + coreNodeName + ", state: " + waitForState
+        + ", checkLive: " + checkLive + ", onlyIfLeader: " + onlyIfLeader + " for: " + retry + " seconds.");
+  }),
 
+  REQUESTRECOVERY_OP(REQUESTRECOVERY, it -> {
+    final SolrParams params = it.req.getParams();
+    log().info("It has been requested that we recover: core=" + params.get(CoreAdminParams.CORE));
+    new Thread(() -> {
       String cname = params.get(CoreAdminParams.CORE);
       if (cname == null) {
-        throw new IllegalArgumentException(CoreAdminParams.CORE + " is required");
+        cname = "";
       }
-
-      SyncStrategy syncStrategy = null;
-      try (SolrCore core = callInfo.handler.coreContainer.getCore(cname)) {
-
+      try (SolrCore core = it.handler.coreContainer.getCore(cname)) {
         if (core != null) {
-          syncStrategy = new SyncStrategy(core.getCoreDescriptor().getCoreContainer());
-
-          Map<String, Object> props = new HashMap<>();
-          props.put(ZkStateReader.BASE_URL_PROP, zkController.getBaseUrl());
-          props.put(ZkStateReader.CORE_NAME_PROP, cname);
-          props.put(ZkStateReader.NODE_NAME_PROP, zkController.getNodeName());
-
-          boolean success = syncStrategy.sync(zkController, core, new ZkNodeProps(props), true);
-          // solrcloud_debug
-          if (log.isDebugEnabled()) {
-            try {
-              RefCounted<SolrIndexSearcher> searchHolder = core
-                  .getNewestSearcher(false);
-              SolrIndexSearcher searcher = searchHolder.get();
-              try {
-                log.debug(core.getCoreDescriptor().getCoreContainer()
-                    .getZkController().getNodeName()
-                    + " synched "
-                    + searcher.search(new MatchAllDocsQuery(), 1).totalHits);
-              } finally {
-                searchHolder.decref();
-              }
-            } catch (Exception e) {
-              log.debug("Error in solrcloud_debug block", e);
-            }
-          }
-          if (!success) {
-            throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Sync Failed");
-          }
+          core.getUpdateHandler().getSolrCoreState().doRecovery(it.handler.coreContainer, core.getCoreDescriptor());
         } else {
-          SolrException.log(log, "Could not find core to call sync:" + cname);
-        }
-      } finally {
-        // no recoveryStrat close for now
-        if (syncStrategy != null) {
-          syncStrategy.close();
+          SolrException.log(log(), "Could not find core to call recovery:" + cname);
         }
       }
+    }).start();
 
+  }),
+  REQUESTSYNCSHARD_OP(REQUESTSYNCSHARD, it -> {
+    final SolrParams params = it.req.getParams();
 
+    log().info("I have been requested to sync up my shard");
+    ZkController zkController = it.handler.coreContainer.getZkController();
+    if (zkController == null) {
+      throw new SolrException(ErrorCode.BAD_REQUEST, "Only valid for SolrCloud");
     }
-  },
-  REQUESTBUFFERUPDATES_OP(REQUESTBUFFERUPDATES) {
-    @Override
-    public void call(CallInfo callInfo) {
-      SolrParams params = callInfo.req.getParams();
-      String cname = params.get(CoreAdminParams.NAME, "");
-      log.info("Starting to buffer updates on core:" + cname);
 
-      try (SolrCore core = callInfo.handler.coreContainer.getCore(cname)) {
-        if (core == null)
-          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Core [" + cname + "] does not exist");
-        UpdateLog updateLog = core.getUpdateHandler().getUpdateLog();
-        if (updateLog.getState() != UpdateLog.State.ACTIVE)  {
-          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Core " + cname + " not in active state");
-        }
-        updateLog.bufferUpdates();
-        callInfo.rsp.add("core", cname);
-        callInfo.rsp.add("status", "BUFFERING");
-      } catch (Throwable e) {
-        if (e instanceof SolrException)
-          throw (SolrException)e;
-        else
-          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Could not start buffering updates", e);
-      } finally {
-        if (callInfo.req != null) callInfo.req.close();
-      }
+    String cname = params.get(CoreAdminParams.CORE);
+    if (cname == null) {
+      throw new IllegalArgumentException(CoreAdminParams.CORE + " is required");
     }
-  },
-  REQUESTAPPLYUPDATES_OP(REQUESTAPPLYUPDATES) {
-    @Override
-    public void call(CallInfo callInfo) {
-      {
-        SolrParams params = callInfo.req.getParams();
-        String cname = params.get(CoreAdminParams.NAME, "");
-        log.info("Applying buffered updates on core: " + cname);
-        CoreContainer coreContainer = callInfo.handler.coreContainer;
-        try (SolrCore core = coreContainer.getCore(cname)) {
-          if (core == null)
-            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Core [" + cname + "] not found");
-          UpdateLog updateLog = core.getUpdateHandler().getUpdateLog();
-          if (updateLog.getState() != UpdateLog.State.BUFFERING) {
-            throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Core " + cname + " not in buffering state");
-          }
-          Future<UpdateLog.RecoveryInfo> future = updateLog.applyBufferedUpdates();
-          if (future == null) {
-            log.info("No buffered updates available. core=" + cname);
-            callInfo.rsp.add("core", cname);
-            callInfo.rsp.add("status", "EMPTY_BUFFER");
-            return;
-          }
-          UpdateLog.RecoveryInfo report = future.get();
-          if (report.failed) {
-            SolrException.log(log, "Replay failed");
-            throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Replay failed");
+
+    SyncStrategy syncStrategy = null;
+    try (SolrCore core = it.handler.coreContainer.getCore(cname)) {
+
+      if (core != null) {
+        syncStrategy = new SyncStrategy(core.getCoreDescriptor().getCoreContainer());
+
+        Map<String, Object> props = new HashMap<>();
+        props.put(ZkStateReader.BASE_URL_PROP, zkController.getBaseUrl());
+        props.put(ZkStateReader.CORE_NAME_PROP, cname);
+        props.put(ZkStateReader.NODE_NAME_PROP, zkController.getNodeName());
+
+        boolean success = syncStrategy.sync(zkController, core, new ZkNodeProps(props), true);
+        // solrcloud_debug
+        if (log().isDebugEnabled()) {
+          try {
+            RefCounted<SolrIndexSearcher> searchHolder = core
+                .getNewestSearcher(false);
+            SolrIndexSearcher searcher = searchHolder.get();
+            try {
+              log().debug(core.getCoreDescriptor().getCoreContainer()
+                  .getZkController().getNodeName()
+                  + " synched "
+                  + searcher.search(new MatchAllDocsQuery(), 1).totalHits);
+            } finally {
+              searchHolder.decref();
+            }
+          } catch (Exception e) {
+            log().debug("Error in solrcloud_debug block", e);
           }
-          coreContainer.getZkController().publish(core.getCoreDescriptor(), Replica.State.ACTIVE);
-          callInfo.rsp.add("core", cname);
-          callInfo.rsp.add("status", "BUFFER_APPLIED");
-        } catch (InterruptedException e) {
-          Thread.currentThread().interrupt();
-          log.warn("Recovery was interrupted", e);
-        } catch (Exception e) {
-          if (e instanceof SolrException)
-            throw (SolrException) e;
-          else
-            throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Could not apply buffered updates", e);
-        } finally {
-          if (callInfo.req != null) callInfo.req.close();
         }
-
-      }
-    }
-  },
-  REQUESTSTATUS_OP(REQUESTSTATUS) {
-    @Override
-    public void call(CallInfo callInfo) {
-      SolrParams params = callInfo.req.getParams();
-      String requestId = params.get(CoreAdminParams.REQUESTID);
-      log.info("Checking request status for : " + requestId);
-
-      if (callInfo.handler.getRequestStatusMap(RUNNING).containsKey(requestId)) {
-        callInfo.rsp.add(RESPONSE_STATUS, RUNNING);
-      } else if (callInfo.handler.getRequestStatusMap(COMPLETED).containsKey(requestId)) {
-        callInfo.rsp.add(RESPONSE_STATUS, COMPLETED);
-        callInfo.rsp.add(RESPONSE, callInfo.handler.getRequestStatusMap(COMPLETED).get(requestId).getRspObject());
-      } else if (callInfo.handler.getRequestStatusMap(FAILED).containsKey(requestId)) {
-        callInfo.rsp.add(RESPONSE_STATUS, FAILED);
-        callInfo.rsp.add(RESPONSE, callInfo.handler.getRequestStatusMap(FAILED).get(requestId).getRspObject());
+        if (!success) {
+          throw new SolrException(ErrorCode.SERVER_ERROR, "Sync Failed");
+        }
       } else {
-        callInfo.rsp.add(RESPONSE_STATUS, "notfound");
-        callInfo.rsp.add(RESPONSE_MESSAGE, "No task found in running, completed or failed tasks");
+        SolrException.log(log(), "Could not find core to call sync:" + cname);
+      }
+    } finally {
+      // no recoveryStrat close for now
+      if (syncStrategy != null) {
+        syncStrategy.close();
       }
-
     }
-  },
-  OVERSEEROP_OP(OVERSEEROP) {
-    @Override
-    public void call(CallInfo callInfo) {
-      ZkController zkController = callInfo.handler.coreContainer.getZkController();
-      if (zkController != null) {
-        String op = callInfo.req.getParams().get("op");
-        String electionNode = callInfo.req.getParams().get("electionNode");
-        if (electionNode != null) {
-          zkController.rejoinOverseerElection(electionNode, "rejoinAtHead".equals(op));
-        } else {
-          log.info("electionNode is required param");
+  }),
+
+  REQUESTBUFFERUPDATES_OP(REQUESTBUFFERUPDATES, it -> {
+    SolrParams params = it.req.getParams();
+    String cname = params.get(CoreAdminParams.NAME, "");
+    log().info("Starting to buffer updates on core:" + cname);
+
+    try (SolrCore core = it.handler.coreContainer.getCore(cname)) {
+      if (core == null)
+        throw new SolrException(ErrorCode.BAD_REQUEST, "Core [" + cname + "] does not exist");
+      UpdateLog updateLog = core.getUpdateHandler().getUpdateLog();
+      if (updateLog.getState() != UpdateLog.State.ACTIVE) {
+        throw new SolrException(ErrorCode.SERVER_ERROR, "Core " + cname + " not in active state");
+      }
+      updateLog.bufferUpdates();
+      it.rsp.add("core", cname);
+      it.rsp.add("status", "BUFFERING");
+    } catch (Throwable e) {
+      if (e instanceof SolrException)
+        throw (SolrException) e;
+      else
+        throw new SolrException(ErrorCode.SERVER_ERROR, "Could not start buffering updates", e);
+    } finally {
+      if (it.req != null) it.req.close();
+    }
+  }),
+  REQUESTAPPLYUPDATES_OP(REQUESTAPPLYUPDATES, it -> {
+    SolrParams params = it.req.getParams();
+    String cname = params.get(CoreAdminParams.NAME, "");
+    log().info("Applying buffered updates on core: " + cname);
+    CoreContainer coreContainer = it.handler.coreContainer;
+    try (SolrCore core = coreContainer.getCore(cname)) {
+      if (core == null)
+        throw new SolrException(ErrorCode.BAD_REQUEST, "Core [" + cname + "] not found");
+      UpdateLog updateLog = core.getUpdateHandler().getUpdateLog();
+      if (updateLog.getState() != UpdateLog.State.BUFFERING) {
+        throw new SolrException(ErrorCode.SERVER_ERROR, "Core " + cname + " not in buffering state");
         }
-      }
+      Future<UpdateLog.RecoveryInfo> future = updateLog.applyBufferedUpdates();
+      if (future == null) {
+        log().info("No buffered updates available. core=" + cname);
+        it.rsp.add("core", cname);
+        it.rsp.add("status", "EMPTY_BUFFER");
+        return;
+      }
+      UpdateLog.RecoveryInfo report = future.get();
+      if (report.failed) {
+        SolrException.log(log(), "Replay failed");
+        throw new SolrException(ErrorCode.SERVER_ERROR, "Replay failed");
+      }
+      coreContainer.getZkController().publish(core.getCoreDescriptor(), Replica.State.ACTIVE);
+      it.rsp.add("core", cname);
+      it.rsp.add("status", "BUFFER_APPLIED");
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      log().warn("Recovery was interrupted", e);
+    } catch (Exception e) {
+      if (e instanceof SolrException)
+        throw (SolrException) e;
+      else
+        throw new SolrException(ErrorCode.SERVER_ERROR, "Could not apply buffered updates", e);
+    } finally {
+      if (it.req != null) it.req.close();
+      }
+  }),
+
+  REQUESTSTATUS_OP(REQUESTSTATUS, it -> {
+    SolrParams params = it.req.getParams();
+    String requestId = params.get(CoreAdminParams.REQUESTID);
+    log().info("Checking request status for : " + requestId);
+
+    if (it.handler.getRequestStatusMap(RUNNING).containsKey(requestId)) {
+      it.rsp.add(RESPONSE_STATUS, RUNNING);
+    } else if (it.handler.getRequestStatusMap(COMPLETED).containsKey(requestId)) {
+      it.rsp.add(RESPONSE_STATUS, COMPLETED);
+      it.rsp.add(RESPONSE, it.handler.getRequestStatusMap(COMPLETED).get(requestId).getRspObject());
+    } else if (it.handler.getRequestStatusMap(FAILED).containsKey(requestId)) {
+      it.rsp.add(RESPONSE_STATUS, FAILED);
+      it.rsp.add(RESPONSE, it.handler.getRequestStatusMap(FAILED).get(requestId).getRspObject());
+    } else {
+      it.rsp.add(RESPONSE_STATUS, "notfound");
+      it.rsp.add(RESPONSE_MESSAGE, "No task found in running, completed or failed tasks");
     }
-  },
-  REJOINLEADERELECTION_OP(REJOINLEADERELECTION) {
-    @Override
-    public void call(CallInfo callInfo) {
-      ZkController zkController = callInfo.handler.coreContainer.getZkController();
-
-      if (zkController != null) {
-        zkController.rejoinShardLeaderElection(callInfo.req.getParams());
+  }),
+
+  OVERSEEROP_OP(OVERSEEROP, it -> {
+    ZkController zkController = it.handler.coreContainer.getZkController();
+    if (zkController != null) {
+      String op = it.req.getParams().get("op");
+      String electionNode = it.req.getParams().get("electionNode");
+      if (electionNode != null) {
+        zkController.rejoinOverseerElection(electionNode, "rejoinAtHead".equals(op));
       } else {
-        log.warn("zkController is null in CoreAdminHandler.handleRequestInternal:REJOINLEADERELECTION. No action taken.");
+        log().info("electionNode is required param");
       }
     }
-  },
-  INVOKE_OP(INVOKE) {
-    @Override
-    public void call(CallInfo callInfo) throws Exception {
-      String[] klas = callInfo.req.getParams().getParams("class");
-      if (klas == null || klas.length == 0) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "class is a required param");
-      }
-      for (String c : klas) {
-        Map<String, Object> result = invokeAClass(callInfo.req, c);
-        callInfo.rsp.add(c, result);
-      }
+  }),
+
+  REJOINLEADERELECTION_OP(REJOINLEADERELECTION, it -> {
+    ZkController zkController = it.handler.coreContainer.getZkController();
 
+    if (zkController != null) {
+      zkController.rejoinShardLeaderElection(it.req.getParams());
+    } else {
+      log().warn("zkController is null in CoreAdminHandler.handleRequestInternal:REJOINLEADERELECTION. No action taken.");
     }
+  }),
 
-    Map<String, Object> invokeAClass(SolrQueryRequest req, String c) {
-      SolrResourceLoader loader = null;
-      if (req.getCore() != null) loader = req.getCore().getResourceLoader();
-      else if (req.getContext().get(CoreContainer.class.getName()) != null) {
-        CoreContainer cc = (CoreContainer) req.getContext().get(CoreContainer.class.getName());
-        loader = cc.getResourceLoader();
-      }
+  INVOKE_OP(INVOKE, it -> {
+    String[] klas = it.req.getParams().getParams("class");
+    if (klas == null || klas.length == 0) {
+      throw new SolrException(ErrorCode.BAD_REQUEST, "class is a required param");
+    }
+    for (String c : klas) {
+      Map<String, Object> result = invokeAClass(it.req, c);
+      it.rsp.add(c, result);
+    }
+  }),
 
-      Invocable invokable = loader.newInstance(c, Invocable.class);
-      Map<String, Object> result = invokable.invoke(req);
-      log.info("Invocable_invoked {}", result);
-      return result;
-    }
-  },
-  FORCEPREPAREFORLEADERSHIP_OP(FORCEPREPAREFORLEADERSHIP) {
-    @Override
-    public void call(CallInfo callInfo) throws IOException {
-      final SolrParams params = callInfo.req.getParams();
-
-      log.info("I have been forcefully prepare myself for leadership.");
-      ZkController zkController = callInfo.handler.coreContainer.getZkController();
-      if (zkController == null) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Only valid for SolrCloud");
-      }
+  FORCEPREPAREFORLEADERSHIP_OP(FORCEPREPAREFORLEADERSHIP, it -> {
+    final SolrParams params = it.req.getParams();
 
-      String cname = params.get(CoreAdminParams.CORE);
-      if (cname == null) {
-        throw new IllegalArgumentException(CoreAdminParams.CORE + " is required");
-      }
-      try (SolrCore core = callInfo.handler.coreContainer.getCore(cname)) {
+    log().info("I have been forcefully prepare myself for leadership.");
+    ZkController zkController = it.handler.coreContainer.getZkController();
+    if (zkController == null) {
+      throw new SolrException(ErrorCode.BAD_REQUEST, "Only valid for SolrCloud");
+    }
 
-        // Setting the last published state for this core to be ACTIVE
-        if (core != null) {
-          core.getCoreDescriptor().getCloudDescriptor().setLastPublished(Replica.State.ACTIVE);
-          log.info("Setting the last published state for this core, {}, to {}", core.getName(), Replica.State.ACTIVE);
-        } else {
-          SolrException.log(log, "Could not find core: " + cname);
-        }
+    String cname = params.get(CoreAdminParams.CORE);
+    if (cname == null) {
+      throw new IllegalArgumentException(CoreAdminParams.CORE + " is required");
+    }
+    try (SolrCore core = it.handler.coreContainer.getCore(cname)) {
+
+      // Setting the last published state for this core to be ACTIVE
+      if (core != null) {
+        core.getCoreDescriptor().getCloudDescriptor().setLastPublished(Replica.State.ACTIVE);
+        log().info("Setting the last published state for this core, {}, to {}", core.getName(), Replica.State.ACTIVE);
+      } else {
+        SolrException.log(log(), "Could not find core: " + cname);
       }
+    }
+  }),
 
+  BACKUPCORE_OP(BACKUPCORE, it -> {
+    ZkController zkController = it.handler.coreContainer.getZkController();
+    if (zkController == null) {
+      throw new SolrException(ErrorCode.BAD_REQUEST, "Internal SolrCloud API");
     }
-  },
-  BACKUPCORE_OP(BACKUPCORE) {
-    @Override
-    public void call(CallInfo callInfo) throws IOException {
-      ZkController zkController = callInfo.handler.coreContainer.getZkController();
-      if (zkController == null) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Internal SolrCloud API");
-      }
 
-      final SolrParams params = callInfo.req.getParams();
-      String cname = params.get(CoreAdminParams.CORE);
-      if (cname == null) {
-        throw new IllegalArgumentException(CoreAdminParams.CORE + " is required");
-      }
+    final SolrParams params = it.req.getParams();
+    String cname = params.get(CoreAdminParams.CORE);
+    if (cname == null) {
+      throw new IllegalArgumentException(CoreAdminParams.CORE + " is required");
+    }
 
-      String name = params.get(NAME);
-      if (name == null) {
-        throw new IllegalArgumentException(CoreAdminParams.NAME + " is required");
-      }
+    String name = params.get(NAME);
+    if (name == null) {
+      throw new IllegalArgumentException(CoreAdminParams.NAME + " is required");
+    }
 
-      String repoName = params.get(CoreAdminParams.BACKUP_REPOSITORY);
-      BackupRepository repository = callInfo.handler.coreContainer.newBackupRepository(Optional.ofNullable(repoName));
+    String repoName = params.get(CoreAdminParams.BACKUP_REPOSITORY);
+    BackupRepository repository = it.handler.coreContainer.newBackupRepository(Optional.ofNullable(repoName));
 
-      String location = repository.getBackupLocation(params.get(CoreAdminParams.BACKUP_LOCATION));
-      if (location == null) {
-        throw new SolrException(ErrorCode.BAD_REQUEST, "'location' is not specified as a query"
-            + " parameter or as a default repository property");
-      }
+    String location = repository.getBackupLocation(params.get(CoreAdminParams.BACKUP_LOCATION));
+    if (location == null) {
+      throw new SolrException(ErrorCode.BAD_REQUEST, "'location' is not specified as a query"
+          + " parameter or as a default repository property");
+    }
 
-      try (SolrCore core = callInfo.handler.coreContainer.getCore(cname)) {
-        SnapShooter snapShooter = new SnapShooter(repository, core, location, name);
-        // validateCreateSnapshot will create parent dirs instead of throw; that choice is dubious.
-        //  But we want to throw. One reason is that
-        //  this dir really should, in fact must, already exist here if triggered via a collection backup on a shared
-        //  file system. Otherwise, perhaps the FS location isn't shared -- we want an error.
-        if (!snapShooter.getBackupRepository().exists(snapShooter.getLocation())) {
-          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-              "Directory to contain snapshots doesn't exist: " + snapShooter.getLocation());
-        }
-        snapShooter.validateCreateSnapshot();
-        snapShooter.createSnapshot();
-      } catch (Exception e) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
-            "Failed to backup core=" + cname + " because " + e, e);
-      }
+    try (SolrCore core = it.handler.coreContainer.getCore(cname)) {
+      SnapShooter snapShooter = new SnapShooter(repository, core, location, name);
+      // validateCreateSnapshot will create parent dirs instead of throw; that choice is dubious.
+      //  But we want to throw. One reason is that
+      //  this dir really should, in fact must, already exist here if triggered via a collection backup on a shared
+      //  file system. Otherwise, perhaps the FS location isn't shared -- we want an error.
+      if (!snapShooter.getBackupRepository().exists(snapShooter.getLocation())) {
+        throw new SolrException(ErrorCode.BAD_REQUEST,
+            "Directory to contain snapshots doesn't exist: " + snapShooter.getLocation());
+      }
+      snapShooter.validateCreateSnapshot();
+      snapShooter.createSnapshot();
+    } catch (Exception e) {
+      throw new SolrException(ErrorCode.SERVER_ERROR,
+          "Failed to backup core=" + cname + " because " + e, e);
     }
-  },
-  RESTORECORE_OP(RESTORECORE) {
-    @Override
-    public void call(CallInfo callInfo) throws Exception {
-      ZkController zkController = callInfo.handler.coreContainer.getZkController();
-      if (zkController == null) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Only valid for SolrCloud");
-      }
+  }),
 
-      final SolrParams params = callInfo.req.getParams();
-      String cname = params.get(CoreAdminParams.CORE);
-      if (cname == null) {
-        throw new IllegalArgumentException(CoreAdminParams.CORE + " is required");
-      }
+  RESTORECORE_OP(RESTORECORE, it -> {
+    ZkController zkController = it.handler.coreContainer.getZkController();
+    if (zkController == null) {
+      throw new SolrException(ErrorCode.BAD_REQUEST, "Only valid for SolrCloud");
+    }
 
-      String name = params.get(NAME);
-      if (name == null) {
-        throw new IllegalArgumentException(CoreAdminParams.NAME + " is required");
-      }
+    final SolrParams params = it.req.getParams();
+    String cname = params.get(CoreAdminParams.CORE);
+    if (cname == null) {
+      throw new IllegalArgumentException(CoreAdminParams.CORE + " is required");
+    }
 
-      String repoName = params.get(CoreAdminParams.BACKUP_REPOSITORY);
-      BackupRepository repository = callInfo.handler.coreContainer.newBackupRepository(Optional.ofNullable(repoName));
+    String name = params.get(NAME);
+    if (name == null) {
+      throw new IllegalArgumentException(CoreAdminParams.NAME + " is required");
+    }
 
-      String location = repository.getBackupLocation(params.get(CoreAdminParams.BACKUP_LOCATION));
-      if (location == null) {
-        throw new SolrException(ErrorCode.BAD_REQUEST, "'location' is not specified as a query"
-            + " parameter or as a default repository property");
-      }
+    String repoName = params.get(CoreAdminParams.BACKUP_REPOSITORY);
+    BackupRepository repository = it.handler.coreContainer.newBackupRepository(Optional.ofNullable(repoName));
 
-      try (SolrCore core = callInfo.handler.coreContainer.getCore(cname)) {
-        RestoreCore restoreCore = new RestoreCore(repository, core, location, name);
-        boolean success = restoreCore.doRestore();
-        if (!success) {
-          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Failed to restore core=" + core.getName());
-        }
+    String location = repository.getBackupLocation(params.get(CoreAdminParams.BACKUP_LOCATION));
+    if (location == null) {
+      throw new SolrException(ErrorCode.BAD_REQUEST, "'location' is not specified as a query"
+          + " parameter or as a default repository property");
+    }
+
+    try (SolrCore core = it.handler.coreContainer.getCore(cname)) {
+      RestoreCore restoreCore = new RestoreCore(repository, core, location, name);
+      boolean success = restoreCore.doRestore();
+      if (!success) {
+        throw new SolrException(ErrorCode.SERVER_ERROR, "Failed to restore core=" + core.getName());
       }
     }
-  };
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  });
 
   final CoreAdminParams.CoreAdminAction action;
+  final CoreAdminOp fun;
 
-  public abstract void call(CallInfo callInfo) throws Exception;
-
-  CoreAdminOperation(CoreAdminParams.CoreAdminAction action) {
+  CoreAdminOperation(CoreAdminParams.CoreAdminAction action, CoreAdminOp fun) {
     this.action = action;
+    this.fun = fun;
+  }
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  static Logger log() {
+    return log;
   }
+
+
+
+
   /**
    * Returns the core status for a particular core.
    * @param cores - the enclosing core container
@@ -937,7 +872,7 @@ enum CoreAdminOperation {
    * @return - a named list of key/value pairs from the core.
    * @throws IOException - LukeRequestHandler can throw an I/O exception
    */
-  NamedList<Object> getCoreStatus(CoreContainer cores, String cname, boolean isIndexInfoNeeded)  throws IOException {
+  static NamedList<Object> getCoreStatus(CoreContainer cores, String cname, boolean isIndexInfoNeeded) throws IOException {
     NamedList<Object> info = new SimpleOrderedMap<>();
 
     if (!cores.isLoaded(cname)) { // Lazily-loaded core, fill in what we can.
@@ -986,7 +921,7 @@ enum CoreAdminOperation {
     return info;
   }
 
-  long getIndexSize(SolrCore core) {
+  static long getIndexSize(SolrCore core) {
     Directory dir;
     long size = 0;
     try {
@@ -1004,4 +939,23 @@ enum CoreAdminOperation {
     return size;
   }
 
+  static Map<String, Object> invokeAClass(SolrQueryRequest req, String c) {
+    SolrResourceLoader loader = null;
+    if (req.getCore() != null) loader = req.getCore().getResourceLoader();
+    else if (req.getContext().get(CoreContainer.class.getName()) != null) {
+      CoreContainer cc = (CoreContainer) req.getContext().get(CoreContainer.class.getName());
+      loader = cc.getResourceLoader();
+    }
+
+    Invocable invokable = loader.newInstance(c, Invocable.class);
+    Map<String, Object> result = invokable.invoke(req);
+    log().info("Invocable_invoked {}", result);
+    return result;
+  }
+
+  @Override
+  public void execute(CallInfo it) throws Exception {
+    fun.execute(it);
+  }
+
 }


[39/40] lucene-solr:apiv2: SOLR-9358: [AngularUI] In Cloud->Tree file view area, collapse metadata by default

Posted by sa...@apache.org.
SOLR-9358: [AngularUI] In Cloud->Tree file view area, collapse metadata by default


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

Branch: refs/heads/apiv2
Commit: d86c369533f0d3fc3cda9d15440521f39d84b750
Parents: 53a34b3
Author: Jan H�ydahl <ja...@apache.org>
Authored: Fri Jul 29 09:48:16 2016 +0200
Committer: Jan H�ydahl <ja...@apache.org>
Committed: Sun Jul 31 23:07:51 2016 +0200

----------------------------------------------------------------------
 solr/CHANGES.txt                      |  2 ++
 solr/webapp/web/css/angular/cloud.css | 14 ++++++++++++++
 solr/webapp/web/partials/cloud.html   |  3 ++-
 3 files changed, 18 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d86c3695/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 8e5b74f..ac4fb21 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -219,6 +219,8 @@ Other Changes
 * SOLR-9340: Change ZooKeeper disconnect and session expiry related logging from INFO to WARN to
   make debugging easier (Varun Thacker)
 
+* SOLR-9358: [AngularUI] In Cloud->Tree file view area, collapse metadata by default (janhoy)
+
 ==================  6.1.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/d86c3695/solr/webapp/web/css/angular/cloud.css
----------------------------------------------------------------------
diff --git a/solr/webapp/web/css/angular/cloud.css b/solr/webapp/web/css/angular/cloud.css
index bbb66ae..4017c22 100644
--- a/solr/webapp/web/css/angular/cloud.css
+++ b/solr/webapp/web/css/angular/cloud.css
@@ -184,6 +184,20 @@ limitations under the License.
   background-image: url( ../../img/ico/cross-1.png );
 }
 
+#content #cloud #file-content #toggle.plus
+{
+  font-style: italic;
+  padding-left: 17px;
+  background-image: url( ../../img/ico/toggle-small-expand.png );
+}
+
+#content #cloud #file-content #toggle.minus
+{
+  font-style: italic;
+  padding-left: 17px;
+  background-image: url( ../../img/ico/toggle-small.png );
+}
+
 #content #cloud #file-content #data
 {
   border-top: 1px solid #c0c0c0;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d86c3695/solr/webapp/web/partials/cloud.html
----------------------------------------------------------------------
diff --git a/solr/webapp/web/partials/cloud.html b/solr/webapp/web/partials/cloud.html
index ef39bb3..415e123 100644
--- a/solr/webapp/web/partials/cloud.html
+++ b/solr/webapp/web/partials/cloud.html
@@ -23,7 +23,8 @@ limitations under the License.
 
       <div id="file-content" class="clearfix">
 
-        <div id="prop" ng-show="znode.prop && showData">
+        <a id="toggle" ng-click="showProps = !showProps" ng-show="showData" ng-class="showProps ? 'minus' : 'plus'">Metadata</a>
+        <div id="prop" ng-show="znode.prop && showData && showProps">
           <ul>
             <li ng-class="{odd:$odd}" ng-repeat="(key, prop) in znode.prop">
               <dl class="clearfix">


[25/40] lucene-solr:apiv2: Add missing update of `lastDocId` in AssertingPointsFormat.

Posted by sa...@apache.org.
Add missing update of `lastDocId` in AssertingPointsFormat.


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

Branch: refs/heads/apiv2
Commit: bc29a3d7ed389485a3a1f9d86017f13b324e73ce
Parents: c2db9fa
Author: Adrien Grand <jp...@gmail.com>
Authored: Thu Jul 28 15:26:51 2016 +0200
Committer: Adrien Grand <jp...@gmail.com>
Committed: Thu Jul 28 15:27:14 2016 +0200

----------------------------------------------------------------------
 .../org/apache/lucene/codecs/asserting/AssertingPointsFormat.java   | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc29a3d7/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java
index c3c672b..0bbf2c6 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java
@@ -126,6 +126,7 @@ public final class AssertingPointsFormat extends PointsFormat {
           assert false: "point values are out of order";
         }
         System.arraycopy(packedValue, 0, lastDocValue, 0, bytesPerDim);
+        lastDocID = docID;
       }
       in.visit(docID, packedValue);
     }


[27/40] lucene-solr:apiv2: LUCENE-7393: restore old myanmar syllable tokenization as an option.

Posted by sa...@apache.org.
LUCENE-7393: restore old myanmar syllable tokenization as an option.


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

Branch: refs/heads/apiv2
Commit: 58f0fbd3767af649da1d47ea62f6f35b1ae28c19
Parents: c9935b4
Author: Robert Muir <rm...@apache.org>
Authored: Thu Jul 28 11:00:24 2016 -0400
Committer: Robert Muir <rm...@apache.org>
Committed: Thu Jul 28 11:00:24 2016 -0400

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   4 +
 .../icu/src/data/uax29/MyanmarSyllable.rbbi     |  50 ++++++
 .../segmentation/DefaultICUTokenizerConfig.java |  14 +-
 .../analysis/icu/segmentation/ICUTokenizer.java |   2 +-
 .../icu/segmentation/ICUTokenizerFactory.java   |   6 +-
 .../icu/segmentation/MyanmarSyllable.brk        | Bin 0 -> 19776 bytes
 .../icu/segmentation/TestICUTokenizer.java      |   6 +-
 .../icu/segmentation/TestICUTokenizerCJK.java   |   2 +-
 .../icu/segmentation/TestMyanmarSyllable.java   | 156 +++++++++++++++++++
 .../segmentation/TestWithCJKBigramFilter.java   |   4 +-
 10 files changed, 234 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/58f0fbd3/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 917dfa2..6958660 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -140,6 +140,10 @@ Improvements
   buffer, instead of a fixed 16.0 MB.  A custom codec can still
   override the buffer size itself. (Mike McCandless)
 
+* LUCENE-7393: Add ICUTokenizer option to parse Myanmar text as syllables instead of words,
+  because the ICU word-breaking algorithm has some issues. This allows for the previous 
+  tokenization used before Lucene 5. (AM, Robert Muir)
+
 Optimizations
 
 * LUCENE-7330, LUCENE-7339: Speed up conjunction queries. (Adrien Grand)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/58f0fbd3/lucene/analysis/icu/src/data/uax29/MyanmarSyllable.rbbi
----------------------------------------------------------------------
diff --git a/lucene/analysis/icu/src/data/uax29/MyanmarSyllable.rbbi b/lucene/analysis/icu/src/data/uax29/MyanmarSyllable.rbbi
new file mode 100644
index 0000000..1840803
--- /dev/null
+++ b/lucene/analysis/icu/src/data/uax29/MyanmarSyllable.rbbi
@@ -0,0 +1,50 @@
+#
+# 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.
+#
+# 
+# Parses Myanmar text, with syllable as token. 
+#
+
+$Cons = [[:Other_Letter:]&[:Myanmar:]];
+$Virama = [\u1039];
+$Asat = [\u103A];
+
+$WordJoin = [:Line_Break=Word_Joiner:]; 
+
+#
+# default numerical definitions
+#
+$Extend       = [\p{Word_Break = Extend}];
+$Format       = [\p{Word_Break = Format}];
+$MidNumLet    = [\p{Word_Break = MidNumLet}];
+$MidNum       = [\p{Word_Break = MidNum}];
+$Numeric      = [\p{Word_Break = Numeric}];
+$ExtendNumLet = [\p{Word_Break = ExtendNumLet}];                                                          
+$MidNumLetEx    = $MidNumLet    ($Extend |  $Format)*;
+$MidNumEx       = $MidNum       ($Extend |  $Format)*;
+$NumericEx      = $Numeric      ($Extend |  $Format)*;
+$ExtendNumLetEx = $ExtendNumLet ($Extend |  $Format)*;
+
+$ConsEx = $Cons ($Extend | $Format)*;
+$AsatEx = $Cons $Asat ($Virama $ConsEx)? ($Extend | $Format)*;
+$MyanmarSyllableEx = $ConsEx ($Virama $ConsEx)? ($AsatEx)*;
+$MyanmarJoinedSyllableEx = $MyanmarSyllableEx ($WordJoin $MyanmarSyllableEx)*;
+
+!!forward;
+$MyanmarJoinedSyllableEx {200};
+
+# default numeric rules
+$NumericEx $ExtendNumLetEx? (($MidNumEx | $MidNumLetEx)? $NumericEx $ExtendNumLetEx?)*  {100};

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/58f0fbd3/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/DefaultICUTokenizerConfig.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/DefaultICUTokenizerConfig.java b/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/DefaultICUTokenizerConfig.java
index b33663d..3cd62c8 100644
--- a/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/DefaultICUTokenizerConfig.java
+++ b/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/DefaultICUTokenizerConfig.java
@@ -63,9 +63,12 @@ public class DefaultICUTokenizerConfig extends ICUTokenizerConfig {
   // the same as ROOT, except no dictionary segmentation for cjk
   private static final BreakIterator defaultBreakIterator = 
     readBreakIterator("Default.brk");
+  private static final BreakIterator myanmarSyllableIterator = 
+    readBreakIterator("MyanmarSyllable.brk");
   
   // TODO: deprecate this boolean? you only care if you are doing super-expert stuff...
   private final boolean cjkAsWords;
+  private final boolean myanmarAsWords;
   
   /** 
    * Creates a new config. This object is lightweight, but the first
@@ -74,9 +77,12 @@ public class DefaultICUTokenizerConfig extends ICUTokenizerConfig {
    *                   otherwise text will be segmented according to UAX#29 defaults.
    *                   If this is true, all Han+Hiragana+Katakana words will be tagged as
    *                   IDEOGRAPHIC.
+   * @param myanmarAsWords true if Myanmar text should undergo dictionary-based segmentation,
+   *                       otherwise it will be tokenized as syllables.
    */
-  public DefaultICUTokenizerConfig(boolean cjkAsWords) { 
+  public DefaultICUTokenizerConfig(boolean cjkAsWords, boolean myanmarAsWords) { 
     this.cjkAsWords = cjkAsWords;
+    this.myanmarAsWords = myanmarAsWords;
   }
   
   @Override
@@ -88,6 +94,12 @@ public class DefaultICUTokenizerConfig extends ICUTokenizerConfig {
   public BreakIterator getBreakIterator(int script) {
     switch(script) {
       case UScript.JAPANESE: return (BreakIterator)cjkBreakIterator.clone();
+      case UScript.MYANMAR: 
+        if (myanmarAsWords) {
+          return (BreakIterator)defaultBreakIterator.clone();
+        } else {
+          return (BreakIterator)myanmarSyllableIterator.clone();
+        }
       default: return (BreakIterator)defaultBreakIterator.clone();
     }
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/58f0fbd3/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/ICUTokenizer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/ICUTokenizer.java b/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/ICUTokenizer.java
index 64c6785..0941551 100644
--- a/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/ICUTokenizer.java
+++ b/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/ICUTokenizer.java
@@ -68,7 +68,7 @@ public final class ICUTokenizer extends Tokenizer {
    * @see DefaultICUTokenizerConfig
    */
   public ICUTokenizer() {
-    this(new DefaultICUTokenizerConfig(true));
+    this(new DefaultICUTokenizerConfig(true, true));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/58f0fbd3/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/ICUTokenizerFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/ICUTokenizerFactory.java b/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/ICUTokenizerFactory.java
index deb5d4f..974e719 100644
--- a/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/ICUTokenizerFactory.java
+++ b/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/ICUTokenizerFactory.java
@@ -79,6 +79,7 @@ public class ICUTokenizerFactory extends TokenizerFactory implements ResourceLoa
   private final Map<Integer,String> tailored;
   private ICUTokenizerConfig config;
   private final boolean cjkAsWords;
+  private final boolean myanmarAsWords;
   
   /** Creates a new ICUTokenizerFactory */
   public ICUTokenizerFactory(Map<String,String> args) {
@@ -95,6 +96,7 @@ public class ICUTokenizerFactory extends TokenizerFactory implements ResourceLoa
       }
     }
     cjkAsWords = getBoolean(args, "cjkAsWords", true);
+    myanmarAsWords = getBoolean(args, "myanmarAsWords", true);
     if (!args.isEmpty()) {
       throw new IllegalArgumentException("Unknown parameters: " + args);
     }
@@ -104,7 +106,7 @@ public class ICUTokenizerFactory extends TokenizerFactory implements ResourceLoa
   public void inform(ResourceLoader loader) throws IOException {
     assert tailored != null : "init must be called first!";
     if (tailored.isEmpty()) {
-      config = new DefaultICUTokenizerConfig(cjkAsWords);
+      config = new DefaultICUTokenizerConfig(cjkAsWords, myanmarAsWords);
     } else {
       final BreakIterator breakers[] = new BreakIterator[UScript.CODE_LIMIT];
       for (Map.Entry<Integer,String> entry : tailored.entrySet()) {
@@ -112,7 +114,7 @@ public class ICUTokenizerFactory extends TokenizerFactory implements ResourceLoa
         String resourcePath = entry.getValue();
         breakers[code] = parseRules(resourcePath, loader);
       }
-      config = new DefaultICUTokenizerConfig(cjkAsWords) {
+      config = new DefaultICUTokenizerConfig(cjkAsWords, myanmarAsWords) {
         
         @Override
         public BreakIterator getBreakIterator(int script) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/58f0fbd3/lucene/analysis/icu/src/resources/org/apache/lucene/analysis/icu/segmentation/MyanmarSyllable.brk
----------------------------------------------------------------------
diff --git a/lucene/analysis/icu/src/resources/org/apache/lucene/analysis/icu/segmentation/MyanmarSyllable.brk b/lucene/analysis/icu/src/resources/org/apache/lucene/analysis/icu/segmentation/MyanmarSyllable.brk
new file mode 100644
index 0000000..41b977b
Binary files /dev/null and b/lucene/analysis/icu/src/resources/org/apache/lucene/analysis/icu/segmentation/MyanmarSyllable.brk differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/58f0fbd3/lucene/analysis/icu/src/test/org/apache/lucene/analysis/icu/segmentation/TestICUTokenizer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/icu/src/test/org/apache/lucene/analysis/icu/segmentation/TestICUTokenizer.java b/lucene/analysis/icu/src/test/org/apache/lucene/analysis/icu/segmentation/TestICUTokenizer.java
index 6398b2c..027baa3 100644
--- a/lucene/analysis/icu/src/test/org/apache/lucene/analysis/icu/segmentation/TestICUTokenizer.java
+++ b/lucene/analysis/icu/src/test/org/apache/lucene/analysis/icu/segmentation/TestICUTokenizer.java
@@ -42,7 +42,7 @@ public class TestICUTokenizer extends BaseTokenStreamTestCase {
     sb.append(whitespace);
     sb.append("testing 1234");
     String input = sb.toString();
-    ICUTokenizer tokenizer = new ICUTokenizer(newAttributeFactory(), new DefaultICUTokenizerConfig(false));
+    ICUTokenizer tokenizer = new ICUTokenizer(newAttributeFactory(), new DefaultICUTokenizerConfig(false, true));
     tokenizer.setReader(new StringReader(input));
     assertTokenStreamContents(tokenizer, new String[] { "testing", "1234" });
   }
@@ -53,7 +53,7 @@ public class TestICUTokenizer extends BaseTokenStreamTestCase {
       sb.append('a');
     }
     String input = sb.toString();
-    ICUTokenizer tokenizer = new ICUTokenizer(newAttributeFactory(), new DefaultICUTokenizerConfig(false));
+    ICUTokenizer tokenizer = new ICUTokenizer(newAttributeFactory(), new DefaultICUTokenizerConfig(false, true));
     tokenizer.setReader(new StringReader(input));
     char token[] = new char[4096];
     Arrays.fill(token, 'a');
@@ -75,7 +75,7 @@ public class TestICUTokenizer extends BaseTokenStreamTestCase {
     a = new Analyzer() {
       @Override
       protected TokenStreamComponents createComponents(String fieldName) {
-        Tokenizer tokenizer = new ICUTokenizer(newAttributeFactory(), new DefaultICUTokenizerConfig(false));
+        Tokenizer tokenizer = new ICUTokenizer(newAttributeFactory(), new DefaultICUTokenizerConfig(false, true));
         TokenFilter filter = new ICUNormalizer2Filter(tokenizer);
         return new TokenStreamComponents(tokenizer, filter);
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/58f0fbd3/lucene/analysis/icu/src/test/org/apache/lucene/analysis/icu/segmentation/TestICUTokenizerCJK.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/icu/src/test/org/apache/lucene/analysis/icu/segmentation/TestICUTokenizerCJK.java b/lucene/analysis/icu/src/test/org/apache/lucene/analysis/icu/segmentation/TestICUTokenizerCJK.java
index a29686c..96f44d6 100644
--- a/lucene/analysis/icu/src/test/org/apache/lucene/analysis/icu/segmentation/TestICUTokenizerCJK.java
+++ b/lucene/analysis/icu/src/test/org/apache/lucene/analysis/icu/segmentation/TestICUTokenizerCJK.java
@@ -34,7 +34,7 @@ public class TestICUTokenizerCJK extends BaseTokenStreamTestCase {
     a = new Analyzer() {
       @Override
       protected TokenStreamComponents createComponents(String fieldName) {
-        return new TokenStreamComponents(new ICUTokenizer(newAttributeFactory(), new DefaultICUTokenizerConfig(true)));
+        return new TokenStreamComponents(new ICUTokenizer(newAttributeFactory(), new DefaultICUTokenizerConfig(true, true)));
       }
     };
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/58f0fbd3/lucene/analysis/icu/src/test/org/apache/lucene/analysis/icu/segmentation/TestMyanmarSyllable.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/icu/src/test/org/apache/lucene/analysis/icu/segmentation/TestMyanmarSyllable.java b/lucene/analysis/icu/src/test/org/apache/lucene/analysis/icu/segmentation/TestMyanmarSyllable.java
new file mode 100644
index 0000000..a3b608e
--- /dev/null
+++ b/lucene/analysis/icu/src/test/org/apache/lucene/analysis/icu/segmentation/TestMyanmarSyllable.java
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.analysis.icu.segmentation;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.Tokenizer;
+
+/** Test tokenizing Myanmar text into syllables */
+public class TestMyanmarSyllable extends BaseTokenStreamTestCase {
+
+  Analyzer a;
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    a = new Analyzer() {
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName) {
+        Tokenizer tokenizer = new ICUTokenizer(newAttributeFactory(), new DefaultICUTokenizerConfig(false, false));
+        return new TokenStreamComponents(tokenizer);
+      }
+    };
+  }
+  
+  @Override
+  public void tearDown() throws Exception {
+    a.close();
+    super.tearDown();
+  }
+  
+  /** as opposed to dictionary break of \u101e\u1000\u103a\u101d\u1004\u103a|\u101c\u103e\u102f\u1015\u103a\u101b\u103e\u102c\u1038|\u1005\u1031|\u1015\u103c\u102e\u1038 */
+  public void testBasics() throws Exception {
+    assertAnalyzesTo(a, "\u101e\u1000\u103a\u101d\u1004\u103a\u101c\u103e\u102f\u1015\u103a\u101b\u103e\u102c\u1038\u1005\u1031\u1015\u103c\u102e\u1038", new String[] { "\u101e\u1000\u103a", "\u101d\u1004\u103a", "\u101c\u103e\u102f\u1015\u103a", "\u101b\u103e\u102c\u1038", "\u1005\u1031", "\u1015\u103c\u102e\u1038" });
+  }
+  
+  // simple tests from "A Rule-based Syllable Segmentation of Myanmar Text" 
+  // * http://www.aclweb.org/anthology/I08-3010
+  // (see also the presentation: http://gii2.nagaokaut.ac.jp/gii/media/share/20080901-ZMM%20Presentation.pdf)
+  // The words are fake, we just test the categories.
+  // note that currently our algorithm is not sophisticated enough to handle some of the special cases!
+  
+  /** constant */
+  public void testC() throws Exception {
+    assertAnalyzesTo(a, "\u1000\u1000", new String[] { "\u1000", "\u1000" });
+  }
+  
+  /** consonant + sign */
+  public void testCF() throws Exception {
+    assertAnalyzesTo(a, "\u1000\u1036\u1000\u1036", new String[] { "\u1000\u1036", "\u1000\u1036" });
+  }
+  
+  /** consonant + consonant + asat */
+  public void testCCA() throws Exception {
+    assertAnalyzesTo(a, "\u1000\u1004\u103a\u1000\u1004\u103a", new String[] { "\u1000\u1004\u103a", "\u1000\u1004\u103a" });
+  }
+  
+  /** consonant + consonant + asat + sign */
+  public void testCCAF() throws Exception {
+    assertAnalyzesTo(a, "\u1000\u1004\u103a\u1038\u1000\u1004\u103a\u1038", new String[] { "\u1000\u1004\u103a\u1038", "\u1000\u1004\u103a\u1038" });
+  }
+  
+  /** consonant + vowel */
+  public void testCV() throws Exception {
+    assertAnalyzesTo(a, "\u1000\u102c\u1000\u102c", new String[] { "\u1000\u102c", "\u1000\u102c" });
+  }
+  
+  /** consonant + vowel + sign */
+  public void testCVF() throws Exception {
+    assertAnalyzesTo(a, "\u1000\u102c\u1038\u1000\u102c\u1038", new String[] { "\u1000\u102c\u1038", "\u1000\u102c\u1038" });
+  }
+  
+  /** consonant + vowel + vowel + asat */
+  public void testCVVA() throws Exception {
+    assertAnalyzesTo(a, "\u1000\u1031\u102c\u103a\u1000\u1031\u102c\u103a", new String[] { "\u1000\u1031\u102c\u103a", "\u1000\u1031\u102c\u103a" });
+  }
+  
+  /** consonant + vowel + vowel + consonant + asat */
+  public void testCVVCA() throws Exception {
+    assertAnalyzesTo(a, "\u1000\u1031\u102c\u1004\u103a\u1000\u1031\u102c\u1004\u103a", new String[] { "\u1000\u1031\u102c\u1004\u103a", "\u1000\u1031\u102c\u1004\u103a" });
+  }
+  
+  /** consonant + vowel + vowel + consonant + asat + sign */
+  public void testCVVCAF() throws Exception {
+    assertAnalyzesTo(a, "\u1000\u1031\u102c\u1004\u103a\u1038\u1000\u1031\u102c\u1004\u103a\u1038", new String[] { "\u1000\u1031\u102c\u1004\u103a\u1038", "\u1000\u1031\u102c\u1004\u103a\u1038" });
+  }
+  
+  /** consonant + medial */
+  public void testCM() throws Exception {
+    assertAnalyzesTo(a, "\u1000\u103b\u1000\u103b", new String[] { "\u1000\u103b", "\u1000\u103b" });
+  }
+  
+  /** consonant + medial + sign */
+  public void testCMF() throws Exception {
+    assertAnalyzesTo(a, "\u1000\u103b\u1036\u1000\u103b\u1036", new String[] { "\u1000\u103b\u1036", "\u1000\u103b\u1036" });
+  }
+  
+  /** consonant + medial + consonant + asat */
+  public void testCMCA() throws Exception {
+    assertAnalyzesTo(a, "\u1000\u103b\u1004\u103a\u1000\u103b\u1004\u103a", new String[] { "\u1000\u103b\u1004\u103a", "\u1000\u103b\u1004\u103a" });
+  }
+  
+  /** consonant + medial + consonant + asat + sign */
+  public void testCMCAF() throws Exception {
+    assertAnalyzesTo(a, "\u1000\u103b\u1004\u103a\u1038\u1000\u103b\u1004\u103a\u1038", new String[] { "\u1000\u103b\u1004\u103a\u1038", "\u1000\u103b\u1004\u103a\u1038" });
+  }
+  
+  /** consonant + medial + vowel */
+  public void testCMV() throws Exception {
+    assertAnalyzesTo(a, "\u1000\u103b\u102c\u1000\u103b\u102c", new String[] { "\u1000\u103b\u102c", "\u1000\u103b\u102c" });
+  }
+  
+  /** consonant + medial + vowel + sign */
+  public void testCMVF() throws Exception {
+    assertAnalyzesTo(a, "\u1000\u103b\u102c\u1038\u1000\u103b\u102c\u1038", new String[] { "\u1000\u103b\u102c\u1038", "\u1000\u103b\u102c\u1038" });
+  }
+  
+  /** consonant + medial + vowel + vowel + asat */
+  public void testCMVVA() throws Exception {
+    assertAnalyzesTo(a, "\u1000\u103b\u1031\u102c\u103a\u1000\u103b\u1031\u102c\u103a", new String[] { "\u1000\u103b\u1031\u102c\u103a", "\u1000\u103b\u1031\u102c\u103a" });
+  }
+  
+  /** consonant + medial + vowel + vowel + consonant + asat */
+  public void testCMVVCA() throws Exception {
+    assertAnalyzesTo(a, "\u1000\u103c\u1031\u102c\u1004\u103a\u1000\u103c\u1031\u102c\u1004\u103a", new String[] { "\u1000\u103c\u1031\u102c\u1004\u103a", "\u1000\u103c\u1031\u102c\u1004\u103a"});
+  }
+  
+  /** consonant + medial + vowel + vowel + consonant + asat + sign */
+  public void testCMVVCAF() throws Exception {
+    assertAnalyzesTo(a, "\u1000\u103c\u1031\u102c\u1004\u103a\u1038\u1000\u103c\u1031\u102c\u1004\u103a\u1038", new String[] { "\u1000\u103c\u1031\u102c\u1004\u103a\u1038", "\u1000\u103c\u1031\u102c\u1004\u103a\u1038"});
+  }
+  
+  /** independent vowel */
+  public void testI() throws Exception {
+    assertAnalyzesTo(a, "\u102a\u102a", new String[] { "\u102a", "\u102a" });
+  }
+  
+  /** independent vowel */
+  public void testE() throws Exception {
+    assertAnalyzesTo(a, "\u1023\u1023", new String[] { "\u1023", "\u1023" });
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/58f0fbd3/lucene/analysis/icu/src/test/org/apache/lucene/analysis/icu/segmentation/TestWithCJKBigramFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/icu/src/test/org/apache/lucene/analysis/icu/segmentation/TestWithCJKBigramFilter.java b/lucene/analysis/icu/src/test/org/apache/lucene/analysis/icu/segmentation/TestWithCJKBigramFilter.java
index 17ea967..411b85e 100644
--- a/lucene/analysis/icu/src/test/org/apache/lucene/analysis/icu/segmentation/TestWithCJKBigramFilter.java
+++ b/lucene/analysis/icu/src/test/org/apache/lucene/analysis/icu/segmentation/TestWithCJKBigramFilter.java
@@ -46,7 +46,7 @@ public class TestWithCJKBigramFilter extends BaseTokenStreamTestCase {
     analyzer = new Analyzer() {
       @Override
       protected TokenStreamComponents createComponents(String fieldName) {
-        Tokenizer source = new ICUTokenizer(newAttributeFactory(), new DefaultICUTokenizerConfig(false));
+        Tokenizer source = new ICUTokenizer(newAttributeFactory(), new DefaultICUTokenizerConfig(false, true));
         TokenStream result = new CJKBigramFilter(source);
         return new TokenStreamComponents(source, new StopFilter(result, CharArraySet.EMPTY_SET));
       }
@@ -60,7 +60,7 @@ public class TestWithCJKBigramFilter extends BaseTokenStreamTestCase {
     analyzer2 = new Analyzer() {
       @Override
       protected TokenStreamComponents createComponents(String fieldName) {
-        Tokenizer source = new ICUTokenizer(newAttributeFactory(), new DefaultICUTokenizerConfig(false));
+        Tokenizer source = new ICUTokenizer(newAttributeFactory(), new DefaultICUTokenizerConfig(false, true));
         // we put this before the CJKBigramFilter, because the normalization might combine
         // some halfwidth katakana forms, which will affect the bigramming.
         TokenStream result = new ICUNormalizer2Filter(source);


[11/40] lucene-solr:apiv2: SOLR-9310: reverting the variable name change becaause it can break back-compat

Posted by sa...@apache.org.
SOLR-9310: reverting the variable name change becaause it can break back-compat


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

Branch: refs/heads/apiv2
Commit: 4c61d5aaccb420a359501612003913fb619e495c
Parents: 8302286
Author: Noble Paul <no...@gmail.com>
Authored: Mon Jul 25 12:29:06 2016 +0530
Committer: Noble Paul <no...@gmail.com>
Committed: Mon Jul 25 12:29:06 2016 +0530

----------------------------------------------------------------------
 .../org/apache/solr/update/IndexFingerprint.java    | 16 +++++++++-------
 1 file changed, 9 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4c61d5aa/solr/core/src/java/org/apache/solr/update/IndexFingerprint.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/IndexFingerprint.java b/solr/core/src/java/org/apache/solr/update/IndexFingerprint.java
index 349b703..b85b224 100644
--- a/solr/core/src/java/org/apache/solr/update/IndexFingerprint.java
+++ b/solr/core/src/java/org/apache/solr/update/IndexFingerprint.java
@@ -42,7 +42,9 @@ public class IndexFingerprint {
 
   private long maxVersionSpecified;
   private long maxVersionEncountered;
-  private long maxVersionsUsedInHash;
+  // this actually means max versions used in computing the hash.
+  // we cannot change this now because it changes back-compat
+  private long maxInHash;
   private long versionsHash;
   private long numVersions;
   private long numDocs;
@@ -56,8 +58,8 @@ public class IndexFingerprint {
     return maxVersionEncountered;
   }
 
-  public long getMaxVersionsUsedInHash() {
-    return maxVersionsUsedInHash;
+  public long getMaxInHash() {
+    return maxInHash;
   }
 
   public long getVersionsHash() {
@@ -113,7 +115,7 @@ public class IndexFingerprint {
         long v = fv.longVal(doc);
         f.maxVersionEncountered = Math.max(v, f.maxVersionEncountered);
         if (v <= f.maxVersionSpecified) {
-          f.maxVersionsUsedInHash = Math.max(v, f.maxVersionsUsedInHash);
+          f.maxInHash = Math.max(v, f.maxInHash);
           f.versionsHash += Hash.fmix64(v);
           f.numVersions++;
         }
@@ -139,7 +141,7 @@ public class IndexFingerprint {
     }
 
     // Go by the highest version under the requested max.
-    cmp = Long.compare(f1.maxVersionsUsedInHash, f2.maxVersionsUsedInHash);
+    cmp = Long.compare(f1.maxInHash, f2.maxInHash);
     if (cmp != 0) return cmp;
 
     // go by who has the most documents in the index
@@ -158,7 +160,7 @@ public class IndexFingerprint {
     Map<String,Object> map = new LinkedHashMap<>();
     map.put("maxVersionSpecified", maxVersionSpecified);
     map.put("maxVersionEncountered", maxVersionEncountered);
-    map.put("maxVersionsUsedInHash", maxVersionsUsedInHash);
+    map.put("maxInHash", maxInHash);
     map.put("versionsHash", versionsHash);
     map.put("numVersions", numVersions);
     map.put("numDocs", numDocs);
@@ -186,7 +188,7 @@ public class IndexFingerprint {
     IndexFingerprint f = new IndexFingerprint();
     f.maxVersionSpecified = getLong(o, "maxVersionSpecified", Long.MAX_VALUE);
     f.maxVersionEncountered = getLong(o, "maxVersionEncountered", -1);
-    f.maxVersionsUsedInHash = getLong(o, "maxVersionsUsedInHash", -1);
+    f.maxInHash = getLong(o, "maxInHash", -1);
     f.versionsHash = getLong(o, "versionsHash", -1);
     f.numVersions = getLong(o, "numVersions", -1);
     f.numDocs = getLong(o, "numDocs", -1);


[21/40] lucene-solr:apiv2: SOLR-9242: Disabling TestLocalFSCloudBackupRestore on Windows till we fix it

Posted by sa...@apache.org.
SOLR-9242: Disabling TestLocalFSCloudBackupRestore on Windows till we fix 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/219406d9
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/219406d9
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/219406d9

Branch: refs/heads/apiv2
Commit: 219406d912e027195145de2e77f35f41a6116c6d
Parents: 3603a62
Author: Varun Thacker <va...@apache.org>
Authored: Wed Jul 27 00:13:36 2016 +0530
Committer: Varun Thacker <va...@apache.org>
Committed: Wed Jul 27 00:13:36 2016 +0530

----------------------------------------------------------------------
 .../test/org/apache/solr/cloud/TestLocalFSCloudBackupRestore.java  | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/219406d9/solr/core/src/test/org/apache/solr/cloud/TestLocalFSCloudBackupRestore.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestLocalFSCloudBackupRestore.java b/solr/core/src/test/org/apache/solr/cloud/TestLocalFSCloudBackupRestore.java
index db68913..c6f6a04 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestLocalFSCloudBackupRestore.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestLocalFSCloudBackupRestore.java
@@ -16,6 +16,7 @@
  */
 package org.apache.solr.cloud;
 
+import org.apache.lucene.util.Constants;
 import org.junit.BeforeClass;
 
 /**
@@ -27,6 +28,7 @@ public class TestLocalFSCloudBackupRestore extends AbstractCloudBackupRestoreTes
 
   @BeforeClass
   public static void setupClass() throws Exception {
+    assumeFalse("Backup/Restore is currently buggy on Windows. Tracking the fix on SOLR-9242", Constants.WINDOWS);
     configureCluster(NUM_SHARDS)// nodes
         .addConfig("conf1", TEST_PATH().resolve("configsets").resolve("cloud-minimal").resolve("conf"))
         .configure();


[18/40] lucene-solr:apiv2: Merge remote-tracking branch 'origin/master'

Posted by sa...@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/aba731a2
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/aba731a2
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/aba731a2

Branch: refs/heads/apiv2
Commit: aba731a28591ef7a41243a260a526eb8fb3b909f
Parents: 9a24dc5 614b45d
Author: Noble Paul <no...@apache.org>
Authored: Tue Jul 26 13:08:44 2016 +0530
Committer: Noble Paul <no...@apache.org>
Committed: Tue Jul 26 13:08:44 2016 +0530

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  5 ++++
 .../simpletext/SimpleTextPointsWriter.java      |  4 +--
 .../org/apache/lucene/codecs/PointsWriter.java  | 11 +++++---
 .../codecs/lucene60/Lucene60PointsWriter.java   | 15 ++++++-----
 .../apache/lucene/index/DocumentsWriter.java    | 11 --------
 .../index/DocumentsWriterFlushControl.java      | 27 ++++++++++++++------
 .../lucene/index/DocumentsWriterPerThread.java  |  2 +-
 .../index/DocumentsWriterStallControl.java      | 26 +++++--------------
 .../org/apache/lucene/index/IndexWriter.java    |  2 +-
 .../lucene/index/LiveIndexWriterConfig.java     | 19 ++++++--------
 .../apache/lucene/index/PointValuesWriter.java  |  6 ++++-
 .../org/apache/lucene/util/bkd/BKDWriter.java   |  2 +-
 .../lucene60/TestLucene60PointsFormat.java      |  5 ++--
 .../index/TestDocumentsWriterStallControl.java  |  6 ++---
 .../apache/lucene/search/TestPointQueries.java  |  5 ++--
 .../apache/lucene/spatial3d/TestGeo3DPoint.java |  5 ++--
 .../codecs/asserting/AssertingPointsFormat.java |  4 +--
 .../codecs/cranky/CrankyPointsFormat.java       |  4 +--
 .../apache/lucene/geo/BaseGeoPointTestCase.java |  3 +--
 .../org/apache/lucene/index/RandomCodec.java    |  9 +++----
 20 files changed, 81 insertions(+), 90 deletions(-)
----------------------------------------------------------------------



[22/40] lucene-solr:apiv2: SOLR-9346: Always close ZkStateReader

Posted by sa...@apache.org.
SOLR-9346: Always close ZkStateReader


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

Branch: refs/heads/apiv2
Commit: 78ebcd3cf5e1106f674f8989958e80d3e37c55bf
Parents: 219406d
Author: Alan Woodward <ro...@apache.org>
Authored: Wed Jul 27 16:45:59 2016 +0100
Committer: Alan Woodward <ro...@apache.org>
Committed: Wed Jul 27 16:45:59 2016 +0100

----------------------------------------------------------------------
 .../apache/solr/hadoop/ZooKeeperInspector.java  |  21 +-
 .../solr/cloud/ChaosMonkeyShardSplitTest.java   |  13 +-
 .../apache/solr/cloud/LeaderElectionTest.java   |   1 +
 .../org/apache/solr/cloud/OverseerTest.java     |   1 +
 .../solr/cloud/overseer/ZkStateWriterTest.java  | 393 ++++++++++---------
 5 files changed, 218 insertions(+), 211 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/78ebcd3c/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/ZooKeeperInspector.java
----------------------------------------------------------------------
diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/ZooKeeperInspector.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/ZooKeeperInspector.java
index 9b86dcd..76928aa 100644
--- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/ZooKeeperInspector.java
+++ b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/ZooKeeperInspector.java
@@ -16,6 +16,15 @@
  */
 package org.apache.solr.hadoop;
 
+import java.io.File;
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
 import com.google.common.io.Files;
 import org.apache.commons.io.FileUtils;
 import org.apache.solr.cloud.ZkController;
@@ -35,15 +44,6 @@ import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.File;
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-
 /**
  * Extracts SolrCloud information from ZooKeeper.
  */
@@ -78,8 +78,7 @@ final class ZooKeeperInspector {
     }
     SolrZkClient zkClient = getZkClient(zkHost);
     
-    try {
-      ZkStateReader zkStateReader = new ZkStateReader(zkClient);
+    try (ZkStateReader zkStateReader = new ZkStateReader(zkClient)) {
       try {
         // first check for alias
         collection = checkForAlias(zkClient, collection);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/78ebcd3c/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyShardSplitTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyShardSplitTest.java b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyShardSplitTest.java
index 190db57..7e840da 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyShardSplitTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyShardSplitTest.java
@@ -16,6 +16,12 @@
  */
 package org.apache.solr.cloud;
 
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+
 import org.apache.lucene.util.LuceneTestCase.Slow;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.common.SolrInputDocument;
@@ -36,12 +42,6 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.util.Collection;
-import java.util.List;
-import java.util.concurrent.atomic.AtomicInteger;
-
 /**
  * Test split phase that occurs when a Collection API split call is made.
  */
@@ -254,6 +254,7 @@ public class ChaosMonkeyShardSplitTest extends ShardSplitTest {
         address.replaceAll("/", "_"));
     overseerElector.setup(ec);
     overseerElector.joinElection(ec, false);
+    reader.close();
     return zkClient;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/78ebcd3c/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java b/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java
index 95dccab..8e1be10 100644
--- a/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java
@@ -118,6 +118,7 @@ public class LeaderElectionTest extends SolrTestCaseJ4 {
       if (!zkClient.isClosed()) {
         zkClient.close();
       }
+      zkStateReader.close();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/78ebcd3c/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java b/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
index 8c9daad..9166a43 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
@@ -130,6 +130,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
         }
       }
       deleteNode(ZkStateReader.LIVE_NODES_ZKNODE + "/" + nodeName);
+      zkStateReader.close();
       zkClient.close();
     }
     

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/78ebcd3c/solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateWriterTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateWriterTest.java b/solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateWriterTest.java
index 86aac4d..85dbf4a 100644
--- a/solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateWriterTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateWriterTest.java
@@ -57,68 +57,69 @@ public class ZkStateWriterTest extends SolrTestCaseJ4 {
       zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
       ZkController.createClusterZkNodes(zkClient);
 
-      ZkStateReader reader = new ZkStateReader(zkClient);
-      reader.createClusterStateWatchersAndUpdate();
-
-      ZkStateWriter writer = new ZkStateWriter(reader, new Overseer.Stats());
-
-      assertFalse("Deletes can always be batched", writer.maybeFlushBefore(new ZkWriteCommand("xyz", null)));
-      assertFalse("Deletes can always be batched", writer.maybeFlushAfter(new ZkWriteCommand("xyz", null)));
-
-      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
-      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c2", true);
-
-      // create new collection with stateFormat = 2
-      ZkWriteCommand c1 = new ZkWriteCommand("c1",
-          new DocCollection("c1", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT, 0, ZkStateReader.COLLECTIONS_ZKNODE + "/c1"));
-      assertFalse("First requests can always be batched", writer.maybeFlushBefore(c1));
-
-      ClusterState clusterState = writer.enqueueUpdate(reader.getClusterState(), c1, null);
-
-      ZkWriteCommand c2 = new ZkWriteCommand("c2",
-          new DocCollection("c2", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT, 0, ZkStateReader.COLLECTIONS_ZKNODE + "/c2"));
-      assertTrue("Different (new) collection create cannot be batched together with another create", writer.maybeFlushBefore(c2));
-
-      // simulate three state changes on same collection, all should be batched together before
-      assertFalse(writer.maybeFlushBefore(c1));
-      assertFalse(writer.maybeFlushBefore(c1));
-      assertFalse(writer.maybeFlushBefore(c1));
-      // and after too
-      assertFalse(writer.maybeFlushAfter(c1));
-      assertFalse(writer.maybeFlushAfter(c1));
-      assertFalse(writer.maybeFlushAfter(c1));
-
-      // simulate three state changes on two different collections with stateFormat=2, none should be batched
-      assertFalse(writer.maybeFlushBefore(c1));
-      // flushAfter has to be called as it updates the internal batching related info
-      assertFalse(writer.maybeFlushAfter(c1));
-      assertTrue(writer.maybeFlushBefore(c2));
-      assertFalse(writer.maybeFlushAfter(c2));
-      assertTrue(writer.maybeFlushBefore(c1));
-      assertFalse(writer.maybeFlushAfter(c1));
-
-      // create a collection in stateFormat = 1 i.e. inside the main cluster state
-      ZkWriteCommand c3 = new ZkWriteCommand("c3",
-          new DocCollection("c3", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT, 0, ZkStateReader.CLUSTER_STATE));
-      clusterState = writer.enqueueUpdate(clusterState, c3, null);
-
-      // simulate three state changes in c3, all should be batched
-      for (int i=0; i<3; i++) {
+      try (ZkStateReader reader = new ZkStateReader(zkClient)) {
+        reader.createClusterStateWatchersAndUpdate();
+
+        ZkStateWriter writer = new ZkStateWriter(reader, new Overseer.Stats());
+
+        assertFalse("Deletes can always be batched", writer.maybeFlushBefore(new ZkWriteCommand("xyz", null)));
+        assertFalse("Deletes can always be batched", writer.maybeFlushAfter(new ZkWriteCommand("xyz", null)));
+
+        zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+        zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c2", true);
+
+        // create new collection with stateFormat = 2
+        ZkWriteCommand c1 = new ZkWriteCommand("c1",
+            new DocCollection("c1", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT, 0, ZkStateReader.COLLECTIONS_ZKNODE + "/c1"));
+        assertFalse("First requests can always be batched", writer.maybeFlushBefore(c1));
+
+        ClusterState clusterState = writer.enqueueUpdate(reader.getClusterState(), c1, null);
+
+        ZkWriteCommand c2 = new ZkWriteCommand("c2",
+            new DocCollection("c2", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT, 0, ZkStateReader.COLLECTIONS_ZKNODE + "/c2"));
+        assertTrue("Different (new) collection create cannot be batched together with another create", writer.maybeFlushBefore(c2));
+
+        // simulate three state changes on same collection, all should be batched together before
+        assertFalse(writer.maybeFlushBefore(c1));
+        assertFalse(writer.maybeFlushBefore(c1));
+        assertFalse(writer.maybeFlushBefore(c1));
+        // and after too
+        assertFalse(writer.maybeFlushAfter(c1));
+        assertFalse(writer.maybeFlushAfter(c1));
+        assertFalse(writer.maybeFlushAfter(c1));
+
+        // simulate three state changes on two different collections with stateFormat=2, none should be batched
+        assertFalse(writer.maybeFlushBefore(c1));
+        // flushAfter has to be called as it updates the internal batching related info
+        assertFalse(writer.maybeFlushAfter(c1));
+        assertTrue(writer.maybeFlushBefore(c2));
+        assertFalse(writer.maybeFlushAfter(c2));
+        assertTrue(writer.maybeFlushBefore(c1));
+        assertFalse(writer.maybeFlushAfter(c1));
+
+        // create a collection in stateFormat = 1 i.e. inside the main cluster state
+        ZkWriteCommand c3 = new ZkWriteCommand("c3",
+            new DocCollection("c3", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT, 0, ZkStateReader.CLUSTER_STATE));
+        clusterState = writer.enqueueUpdate(clusterState, c3, null);
+
+        // simulate three state changes in c3, all should be batched
+        for (int i = 0; i < 3; i++) {
+          assertFalse(writer.maybeFlushBefore(c3));
+          assertFalse(writer.maybeFlushAfter(c3));
+        }
+
+        // simulate state change in c3 (stateFormat=1) interleaved with state changes from c1,c2 (stateFormat=2)
+        // none should be batched together
         assertFalse(writer.maybeFlushBefore(c3));
         assertFalse(writer.maybeFlushAfter(c3));
+        assertTrue("different stateFormat, should be flushed", writer.maybeFlushBefore(c1));
+        assertFalse(writer.maybeFlushAfter(c1));
+        assertTrue("different stateFormat, should be flushed", writer.maybeFlushBefore(c3));
+        assertFalse(writer.maybeFlushAfter(c3));
+        assertTrue("different stateFormat, should be flushed", writer.maybeFlushBefore(c2));
+        assertFalse(writer.maybeFlushAfter(c2));
       }
 
-      // simulate state change in c3 (stateFormat=1) interleaved with state changes from c1,c2 (stateFormat=2)
-      // none should be batched together
-      assertFalse(writer.maybeFlushBefore(c3));
-      assertFalse(writer.maybeFlushAfter(c3));
-      assertTrue("different stateFormat, should be flushed", writer.maybeFlushBefore(c1));
-      assertFalse(writer.maybeFlushAfter(c1));
-      assertTrue("different stateFormat, should be flushed", writer.maybeFlushBefore(c3));
-      assertFalse(writer.maybeFlushAfter(c3));
-      assertTrue("different stateFormat, should be flushed", writer.maybeFlushBefore(c2));
-      assertFalse(writer.maybeFlushAfter(c2));
-
     } finally {
       IOUtils.close(zkClient);
       server.shutdown();
@@ -140,24 +141,25 @@ public class ZkStateWriterTest extends SolrTestCaseJ4 {
       zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
       ZkController.createClusterZkNodes(zkClient);
 
-      ZkStateReader reader = new ZkStateReader(zkClient);
-      reader.createClusterStateWatchersAndUpdate();
+      try (ZkStateReader reader = new ZkStateReader(zkClient)) {
+        reader.createClusterStateWatchersAndUpdate();
 
-      ZkStateWriter writer = new ZkStateWriter(reader, new Overseer.Stats());
+        ZkStateWriter writer = new ZkStateWriter(reader, new Overseer.Stats());
 
-      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+        zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
 
-      // create new collection with stateFormat = 1
-      ZkWriteCommand c1 = new ZkWriteCommand("c1",
-          new DocCollection("c1", new HashMap<String, Slice>(), new HashMap<String, Object>(), DocRouter.DEFAULT, 0, ZkStateReader.CLUSTER_STATE));
+        // create new collection with stateFormat = 1
+        ZkWriteCommand c1 = new ZkWriteCommand("c1",
+            new DocCollection("c1", new HashMap<String, Slice>(), new HashMap<String, Object>(), DocRouter.DEFAULT, 0, ZkStateReader.CLUSTER_STATE));
 
-      ClusterState clusterState = writer.enqueueUpdate(reader.getClusterState(), c1, null);
-      writer.writePendingUpdates();
+        ClusterState clusterState = writer.enqueueUpdate(reader.getClusterState(), c1, null);
+        writer.writePendingUpdates();
 
-      Map map = (Map) Utils.fromJSON(zkClient.getData("/clusterstate.json", null, null, true));
-      assertNotNull(map.get("c1"));
-      boolean exists = zkClient.exists(ZkStateReader.COLLECTIONS_ZKNODE + "/c1/state.json", true);
-      assertFalse(exists);
+        Map map = (Map) Utils.fromJSON(zkClient.getData("/clusterstate.json", null, null, true));
+        assertNotNull(map.get("c1"));
+        boolean exists = zkClient.exists(ZkStateReader.COLLECTIONS_ZKNODE + "/c1/state.json", true);
+        assertFalse(exists);
+      }
 
     } finally {
       IOUtils.close(zkClient);
@@ -181,24 +183,25 @@ public class ZkStateWriterTest extends SolrTestCaseJ4 {
       zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
       ZkController.createClusterZkNodes(zkClient);
 
-      ZkStateReader reader = new ZkStateReader(zkClient);
-      reader.createClusterStateWatchersAndUpdate();
+      try (ZkStateReader reader = new ZkStateReader(zkClient)) {
+        reader.createClusterStateWatchersAndUpdate();
 
-      ZkStateWriter writer = new ZkStateWriter(reader, new Overseer.Stats());
+        ZkStateWriter writer = new ZkStateWriter(reader, new Overseer.Stats());
 
-      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+        zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
 
-      // create new collection with stateFormat = 2
-      ZkWriteCommand c1 = new ZkWriteCommand("c1",
-          new DocCollection("c1", new HashMap<String, Slice>(), new HashMap<String, Object>(), DocRouter.DEFAULT, 0, ZkStateReader.COLLECTIONS_ZKNODE + "/c1/state.json"));
+        // create new collection with stateFormat = 2
+        ZkWriteCommand c1 = new ZkWriteCommand("c1",
+            new DocCollection("c1", new HashMap<String, Slice>(), new HashMap<String, Object>(), DocRouter.DEFAULT, 0, ZkStateReader.COLLECTIONS_ZKNODE + "/c1/state.json"));
 
-      ClusterState clusterState = writer.enqueueUpdate(reader.getClusterState(), c1, null);
-      writer.writePendingUpdates();
+        ClusterState clusterState = writer.enqueueUpdate(reader.getClusterState(), c1, null);
+        writer.writePendingUpdates();
 
-      Map map = (Map) Utils.fromJSON(zkClient.getData("/clusterstate.json", null, null, true));
-      assertNull(map.get("c1"));
-      map = (Map) Utils.fromJSON(zkClient.getData(ZkStateReader.COLLECTIONS_ZKNODE + "/c1/state.json", null, null, true));
-      assertNotNull(map.get("c1"));
+        Map map = (Map) Utils.fromJSON(zkClient.getData("/clusterstate.json", null, null, true));
+        assertNull(map.get("c1"));
+        map = (Map) Utils.fromJSON(zkClient.getData(ZkStateReader.COLLECTIONS_ZKNODE + "/c1/state.json", null, null, true));
+        assertNotNull(map.get("c1"));
+      }
 
     } finally {
       IOUtils.close(zkClient);
@@ -224,63 +227,64 @@ public class ZkStateWriterTest extends SolrTestCaseJ4 {
       zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
       ZkController.createClusterZkNodes(zkClient);
 
-      ZkStateReader reader = new ZkStateReader(zkClient);
-      reader.createClusterStateWatchersAndUpdate();
-
-      ZkStateWriter writer = new ZkStateWriter(reader, new Overseer.Stats());
-
-      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
-      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c2", true);
-
-      // create collection 1 with stateFormat = 1
-      ZkWriteCommand c1 = new ZkWriteCommand("c1",
-          new DocCollection("c1", new HashMap<String, Slice>(), new HashMap<String, Object>(), DocRouter.DEFAULT, 0, ZkStateReader.CLUSTER_STATE));
-      writer.enqueueUpdate(reader.getClusterState(), c1, null);
-      writer.writePendingUpdates();
+      try (ZkStateReader reader = new ZkStateReader(zkClient)) {
+        reader.createClusterStateWatchersAndUpdate();
 
-      reader.forceUpdateCollection("c1");
-      reader.forceUpdateCollection("c2");
-      ClusterState clusterState = reader.getClusterState(); // keep a reference to the current cluster state object
-      assertTrue(clusterState.hasCollection("c1"));
-      assertFalse(clusterState.hasCollection("c2"));
+        ZkStateWriter writer = new ZkStateWriter(reader, new Overseer.Stats());
 
-      // Simulate an external modification to /clusterstate.json
-      byte[] data = zkClient.getData("/clusterstate.json", null, null, true);
-      zkClient.setData("/clusterstate.json", data, true);
+        zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+        zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c2", true);
 
-      // enqueue another c1 so that ZkStateWriter has pending updates
-      writer.enqueueUpdate(clusterState, c1, null);
-      assertTrue(writer.hasPendingUpdates());
-
-      // create collection 2 with stateFormat = 1
-      ZkWriteCommand c2 = new ZkWriteCommand("c2",
-          new DocCollection("c2", new HashMap<String, Slice>(), new HashMap<String, Object>(), DocRouter.DEFAULT, 0, ZkStateReader.getCollectionPath("c2")));
-
-      try {
-        writer.enqueueUpdate(clusterState, c2, null); // we are sending in the old cluster state object
-        fail("Enqueue should not have succeeded");
-      } catch (KeeperException.BadVersionException bve) {
-        // expected
-      }
-
-      try {
-        writer.enqueueUpdate(reader.getClusterState(), c2, null);
-        fail("enqueueUpdate after BadVersionException should not have succeeded");
-      } catch (IllegalStateException e) {
-        // expected
-      }
-
-      try {
+        // create collection 1 with stateFormat = 1
+        ZkWriteCommand c1 = new ZkWriteCommand("c1",
+            new DocCollection("c1", new HashMap<String, Slice>(), new HashMap<String, Object>(), DocRouter.DEFAULT, 0, ZkStateReader.CLUSTER_STATE));
+        writer.enqueueUpdate(reader.getClusterState(), c1, null);
         writer.writePendingUpdates();
-        fail("writePendingUpdates after BadVersionException should not have succeeded");
-      } catch (IllegalStateException e) {
-        // expected
-      }
 
+        reader.forceUpdateCollection("c1");
+        reader.forceUpdateCollection("c2");
+        ClusterState clusterState = reader.getClusterState(); // keep a reference to the current cluster state object
+        assertTrue(clusterState.hasCollection("c1"));
+        assertFalse(clusterState.hasCollection("c2"));
+
+        // Simulate an external modification to /clusterstate.json
+        byte[] data = zkClient.getData("/clusterstate.json", null, null, true);
+        zkClient.setData("/clusterstate.json", data, true);
+
+        // enqueue another c1 so that ZkStateWriter has pending updates
+        writer.enqueueUpdate(clusterState, c1, null);
+        assertTrue(writer.hasPendingUpdates());
+
+        // create collection 2 with stateFormat = 1
+        ZkWriteCommand c2 = new ZkWriteCommand("c2",
+            new DocCollection("c2", new HashMap<String, Slice>(), new HashMap<String, Object>(), DocRouter.DEFAULT, 0, ZkStateReader.getCollectionPath("c2")));
+
+        try {
+          writer.enqueueUpdate(clusterState, c2, null); // we are sending in the old cluster state object
+          fail("Enqueue should not have succeeded");
+        } catch (KeeperException.BadVersionException bve) {
+          // expected
+        }
+
+        try {
+          writer.enqueueUpdate(reader.getClusterState(), c2, null);
+          fail("enqueueUpdate after BadVersionException should not have succeeded");
+        } catch (IllegalStateException e) {
+          // expected
+        }
+
+        try {
+          writer.writePendingUpdates();
+          fail("writePendingUpdates after BadVersionException should not have succeeded");
+        } catch (IllegalStateException e) {
+          // expected
+        }
+      }
     } finally {
       IOUtils.close(zkClient);
       server.shutdown();
     }
+
   }
 
   public void testExternalModificationToStateFormat2() throws Exception {
@@ -298,68 +302,69 @@ public class ZkStateWriterTest extends SolrTestCaseJ4 {
       zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
       ZkController.createClusterZkNodes(zkClient);
 
-      ZkStateReader reader = new ZkStateReader(zkClient);
-      reader.createClusterStateWatchersAndUpdate();
-
-      ZkStateWriter writer = new ZkStateWriter(reader, new Overseer.Stats());
-
-      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
-      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c2", true);
-
-      ClusterState state = reader.getClusterState();
-
-      // create collection 2 with stateFormat = 2
-      ZkWriteCommand c2 = new ZkWriteCommand("c2",
-          new DocCollection("c2", new HashMap<String, Slice>(), new HashMap<String, Object>(), DocRouter.DEFAULT, 0, ZkStateReader.getCollectionPath("c2")));
-      state = writer.enqueueUpdate(reader.getClusterState(), c2, null);
-      assertFalse(writer.hasPendingUpdates()); // first write is flushed immediately
-
-      int sharedClusterStateVersion = state.getZkClusterStateVersion();
-      int stateFormat2Version = state.getCollection("c2").getZNodeVersion();
-
-      // Simulate an external modification to /collections/c2/state.json
-      byte[] data = zkClient.getData(ZkStateReader.getCollectionPath("c2"), null, null, true);
-      zkClient.setData(ZkStateReader.getCollectionPath("c2"), data, true);
-
-      // get the most up-to-date state
-      reader.forceUpdateCollection("c2");
-      state = reader.getClusterState();
-      log.info("Cluster state: {}", state);
-      assertTrue(state.hasCollection("c2"));
-      assertEquals(sharedClusterStateVersion, (int) state.getZkClusterStateVersion());
-      assertEquals(stateFormat2Version + 1, state.getCollection("c2").getZNodeVersion());
-
-      // enqueue an update to stateFormat2 collection such that update is pending
-      state = writer.enqueueUpdate(state, c2, null);
-      assertTrue(writer.hasPendingUpdates());
-
-      // get the most up-to-date state
-      reader.forceUpdateCollection("c2");
-      state = reader.getClusterState();
-
-      // enqueue a stateFormat=1 collection which should cause a flush
-      ZkWriteCommand c1 = new ZkWriteCommand("c1",
-          new DocCollection("c1", new HashMap<String, Slice>(), new HashMap<String, Object>(), DocRouter.DEFAULT, 0, ZkStateReader.CLUSTER_STATE));
-
-      try {
-        writer.enqueueUpdate(state, c1, null);
-        fail("Enqueue should not have succeeded");
-      } catch (KeeperException.BadVersionException bve) {
-        // expected
-      }
-
-      try {
-        writer.enqueueUpdate(reader.getClusterState(), c2, null);
-        fail("enqueueUpdate after BadVersionException should not have succeeded");
-      } catch (IllegalStateException e) {
-        // expected
-      }
-
-      try {
-        writer.writePendingUpdates();
-        fail("writePendingUpdates after BadVersionException should not have succeeded");
-      } catch (IllegalStateException e) {
-        // expected
+      try (ZkStateReader reader = new ZkStateReader(zkClient)) {
+        reader.createClusterStateWatchersAndUpdate();
+
+        ZkStateWriter writer = new ZkStateWriter(reader, new Overseer.Stats());
+
+        zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+        zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c2", true);
+
+        ClusterState state = reader.getClusterState();
+
+        // create collection 2 with stateFormat = 2
+        ZkWriteCommand c2 = new ZkWriteCommand("c2",
+            new DocCollection("c2", new HashMap<String, Slice>(), new HashMap<String, Object>(), DocRouter.DEFAULT, 0, ZkStateReader.getCollectionPath("c2")));
+        state = writer.enqueueUpdate(reader.getClusterState(), c2, null);
+        assertFalse(writer.hasPendingUpdates()); // first write is flushed immediately
+
+        int sharedClusterStateVersion = state.getZkClusterStateVersion();
+        int stateFormat2Version = state.getCollection("c2").getZNodeVersion();
+
+        // Simulate an external modification to /collections/c2/state.json
+        byte[] data = zkClient.getData(ZkStateReader.getCollectionPath("c2"), null, null, true);
+        zkClient.setData(ZkStateReader.getCollectionPath("c2"), data, true);
+
+        // get the most up-to-date state
+        reader.forceUpdateCollection("c2");
+        state = reader.getClusterState();
+        log.info("Cluster state: {}", state);
+        assertTrue(state.hasCollection("c2"));
+        assertEquals(sharedClusterStateVersion, (int) state.getZkClusterStateVersion());
+        assertEquals(stateFormat2Version + 1, state.getCollection("c2").getZNodeVersion());
+
+        // enqueue an update to stateFormat2 collection such that update is pending
+        state = writer.enqueueUpdate(state, c2, null);
+        assertTrue(writer.hasPendingUpdates());
+
+        // get the most up-to-date state
+        reader.forceUpdateCollection("c2");
+        state = reader.getClusterState();
+
+        // enqueue a stateFormat=1 collection which should cause a flush
+        ZkWriteCommand c1 = new ZkWriteCommand("c1",
+            new DocCollection("c1", new HashMap<String, Slice>(), new HashMap<String, Object>(), DocRouter.DEFAULT, 0, ZkStateReader.CLUSTER_STATE));
+
+        try {
+          writer.enqueueUpdate(state, c1, null);
+          fail("Enqueue should not have succeeded");
+        } catch (KeeperException.BadVersionException bve) {
+          // expected
+        }
+
+        try {
+          writer.enqueueUpdate(reader.getClusterState(), c2, null);
+          fail("enqueueUpdate after BadVersionException should not have succeeded");
+        } catch (IllegalStateException e) {
+          // expected
+        }
+
+        try {
+          writer.writePendingUpdates();
+          fail("writePendingUpdates after BadVersionException should not have succeeded");
+        } catch (IllegalStateException e) {
+          // expected
+        }
       }
     } finally {
       IOUtils.close(zkClient);


[24/40] lucene-solr:apiv2: SOLR-9106: Add javadocs to ZkStateReader cluster properties methods

Posted by sa...@apache.org.
SOLR-9106: Add javadocs to ZkStateReader cluster properties methods


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

Branch: refs/heads/apiv2
Commit: c2db9fae2cc312a13a66e6dab9989ed65738fe02
Parents: ef53ef9
Author: Alan Woodward <ro...@apache.org>
Authored: Thu Jul 28 09:34:38 2016 +0100
Committer: Alan Woodward <ro...@apache.org>
Committed: Thu Jul 28 10:55:52 2016 +0100

----------------------------------------------------------------------
 .../apache/solr/common/cloud/ZkStateReader.java | 21 ++++++++++++++++++++
 1 file changed, 21 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c2db9fae/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
index 227a8b3..9928346 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
@@ -825,6 +825,18 @@ public class ZkStateReader implements Closeable {
     this.aliases = ClusterState.load(data);
   }
 
+  /**
+   * Get a cluster property
+   *
+   * N.B. Cluster properties are updated via ZK watchers, and so may not necessarily
+   * be completely up-to-date.  If you need to get the latest version, then use a
+   * {@link ClusterProperties} instance.
+   *
+   * @param key           the property to read
+   * @param defaultValue  a default value to use if no such property exists
+   * @param <T>           the type of the property
+   * @return the cluster property, or a default if the property is not set
+   */
   @SuppressWarnings("unchecked")
   public <T> T getClusterProperty(String key, T defaultValue) {
     T value = (T) clusterProperties.get(key);
@@ -833,6 +845,15 @@ public class ZkStateReader implements Closeable {
     return value;
   }
 
+  /**
+   * Get all cluster properties for this cluster
+   *
+   * N.B. Cluster properties are updated via ZK watchers, and so may not necessarily
+   * be completely up-to-date.  If you need to get the latest version, then use a
+   * {@link ClusterProperties} instance.
+   *
+   * @return a Map of cluster properties
+   */
   public Map<String, Object> getClusterProperties() {
     return Collections.unmodifiableMap(clusterProperties);
   }


[29/40] lucene-solr:apiv2: SOLR-9200: Add Delegation Token Support to Solr

Posted by sa...@apache.org.
SOLR-9200: Add Delegation Token Support to 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/7bf019a9
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/7bf019a9
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/7bf019a9

Branch: refs/heads/apiv2
Commit: 7bf019a9c2779368133850f141174febede6ad96
Parents: 58f0fbd
Author: Gregory Chanan <gc...@cloudera.com>
Authored: Fri Jun 17 16:49:48 2016 -0700
Committer: Gregory Chanan <gc...@cloudera.com>
Committed: Thu Jul 28 12:59:05 2016 -0400

----------------------------------------------------------------------
 lucene/ivy-versions.properties                  |   1 +
 solr/CHANGES.txt                                |   3 +
 solr/core/ivy.xml                               |   4 +
 .../solr/security/AuthenticationPlugin.java     |  29 +-
 .../apache/solr/security/BasicAuthPlugin.java   |   5 +-
 .../security/DelegationTokenKerberosFilter.java | 171 ++++++++
 .../apache/solr/security/KerberosFilter.java    |  14 +
 .../apache/solr/security/KerberosPlugin.java    | 195 ++++++++-
 .../solr/security/PKIAuthenticationPlugin.java  |  13 +-
 .../solr/security/PrintWriterWrapper.java       | 215 ++++++++++
 .../apache/solr/servlet/SolrDispatchFilter.java |  12 +-
 .../apache/solr/cloud/KerberosTestServices.java | 228 +++++++++++
 .../org/apache/solr/cloud/KerberosTestUtil.java | 147 -------
 ...utOfBoxZkACLAndCredentialsProvidersTest.java |   7 +-
 ...rriddenZkACLAndCredentialsProvidersTest.java |  71 ++--
 .../solr/cloud/SaslZkACLProviderTest.java       |  39 +-
 .../solr/cloud/TestAuthenticationFramework.java |  10 +-
 .../cloud/TestMiniSolrCloudClusterKerberos.java |  29 +-
 .../TestSolrCloudWithDelegationTokens.java      | 405 +++++++++++++++++++
 .../cloud/TestSolrCloudWithKerberosAlt.java     |  37 +-
 ...MParamsZkACLAndCredentialsProvidersTest.java |  25 +-
 ...ramDelegationTokenAuthenticationHandler.java | 109 +++++
 .../solr/security/MockAuthenticationPlugin.java |  32 +-
 solr/licenses/curator-recipes-2.8.0.jar.sha1    |   1 +
 solr/licenses/curator-recipes-LICENSE-ASL.txt   | 202 +++++++++
 solr/licenses/curator-recipes-NOTICE.txt        |   5 +
 solr/solrj/ivy.xml                              |   5 +
 .../solr/client/solrj/impl/HttpSolrClient.java  |  57 ++-
 .../solrj/impl/Krb5HttpClientBuilder.java       |  18 +-
 .../solrj/request/DelegationTokenRequest.java   | 152 +++++++
 .../solrj/response/DelegationTokenResponse.java | 108 +++++
 .../solr/common/cloud/SaslZkACLProvider.java    |  21 +-
 .../cloud/SecurityAwareZkACLProvider.java       |  79 ++++
 .../apache/solr/common/cloud/SolrZkClient.java  |   2 +-
 ...ParamsAllAndReadonlyDigestZkACLProvider.java |  52 ++-
 .../cloud/ZkClientConnectionStrategy.java       |   4 +-
 .../request/TestDelegationTokenRequest.java     |  70 ++++
 .../response/TestDelegationTokenResponse.java   | 138 +++++++
 38 files changed, 2376 insertions(+), 339 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/lucene/ivy-versions.properties
----------------------------------------------------------------------
diff --git a/lucene/ivy-versions.properties b/lucene/ivy-versions.properties
index 510befa..bc46ee6 100644
--- a/lucene/ivy-versions.properties
+++ b/lucene/ivy-versions.properties
@@ -106,6 +106,7 @@ io.netty.netty-all.version = 4.0.36.Final
 org.apache.curator.version = 2.8.0
 /org.apache.curator/curator-client = ${org.apache.curator.version}
 /org.apache.curator/curator-framework = ${org.apache.curator.version}
+/org.apache.curator/curator-recipes = ${org.apache.curator.version}
 
 /org.apache.derby/derby = 10.9.1.0
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index a728c8a..6707e1a 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -100,6 +100,9 @@ New Features
 * SOLR-9275: XML QueryParser support (defType=xmlparser) now extensible via configuration.
   (Christine Poerschke)
 
+* SOLR-9200: Add Delegation Token Support to Solr.
+  (Gregory Chanan)
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/core/ivy.xml
----------------------------------------------------------------------
diff --git a/solr/core/ivy.xml b/solr/core/ivy.xml
index 5dad49b..08272ad 100644
--- a/solr/core/ivy.xml
+++ b/solr/core/ivy.xml
@@ -134,6 +134,10 @@
     <dependency org="antlr" name="antlr" rev="${/antlr/antlr}" conf="test.MiniKdc"/>
     <dependency org="net.sf.ehcache" name="ehcache-core" rev="${/net.sf.ehcache/ehcache-core}" conf="test.MiniKdc"/>
 
+    <dependency org="org.apache.curator" name="curator-framework" rev="${/org.apache.curator/curator-framework}" conf="compile"/>
+    <dependency org="org.apache.curator" name="curator-client" rev="${/org.apache.curator/curator-client}" conf="compile"/>
+    <dependency org="org.apache.curator" name="curator-recipes" rev="${/org.apache.curator/curator-recipes}" conf="compile"/>
+
     <!-- StatsComponents percentiles Dependencies-->
     <dependency org="com.tdunning" name="t-digest" rev="${/com.tdunning/t-digest}" conf="compile->*"/>
     <!-- SQL Parser -->

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java b/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java
index 105f307..d8f2ef2 100644
--- a/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java
@@ -17,18 +17,11 @@
 package org.apache.solr.security;
 
 import javax.servlet.FilterChain;
-import javax.servlet.ServletException;
 import javax.servlet.ServletRequest;
 import javax.servlet.ServletResponse;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletRequestWrapper;
 import java.io.Closeable;
-import java.io.IOException;
-import java.security.Principal;
 import java.util.Map;
 
-import org.apache.http.auth.BasicUserPrincipal;
-
 /**
  * 
  * @lucene.experimental
@@ -42,32 +35,20 @@ public abstract class AuthenticationPlugin implements Closeable {
    * @param pluginConfig Config parameters, possibly from a ZK source
    */
   public abstract void init(Map<String, Object> pluginConfig);
-
-  protected void forward(String user, ServletRequest  req, ServletResponse rsp,
-                                    FilterChain chain) throws IOException, ServletException {
-    if(user != null) {
-      final Principal p = new BasicUserPrincipal(user);
-      req = new HttpServletRequestWrapper((HttpServletRequest) req) {
-        @Override
-        public Principal getUserPrincipal() {
-          return p;
-        }
-      };
-    }
-    chain.doFilter(req,rsp);
-  }
  
   /**
-   * This method must authenticate the request. Upon a successful authentication, this 
+   * This method attempts to authenticate the request. Upon a successful authentication, this
    * must call the next filter in the filter chain and set the user principal of the request,
    * or else, upon an error or an authentication failure, throw an exception.
-   * 
+   *
    * @param request the http request
    * @param response the http response
    * @param filterChain the servlet filter chain
+   * @return false if the request not be processed by Solr (not continue), i.e.
+   * the response and status code have already been sent.
    * @throws Exception any exception thrown during the authentication, e.g. PrivilegedActionException
    */
-  public abstract void doAuthenticate(ServletRequest request, ServletResponse response,
+  public abstract boolean doAuthenticate(ServletRequest request, ServletResponse response,
       FilterChain filterChain) throws Exception;
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java b/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java
index 03c75c6..e3f53a2 100644
--- a/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java
@@ -99,7 +99,7 @@ public class BasicAuthPlugin extends AuthenticationPlugin implements ConfigEdita
   }
 
   @Override
-  public void doAuthenticate(ServletRequest servletRequest, ServletResponse servletResponse, FilterChain filterChain) throws Exception {
+  public boolean doAuthenticate(ServletRequest servletRequest, ServletResponse servletResponse, FilterChain filterChain) throws Exception {
 
     HttpServletRequest request = (HttpServletRequest) servletRequest;
     HttpServletResponse response = (HttpServletResponse) servletResponse;
@@ -127,6 +127,7 @@ public class BasicAuthPlugin extends AuthenticationPlugin implements ConfigEdita
                   }
                 };
                 filterChain.doFilter(wrapper, response);
+                return true;
               }
 
             } else {
@@ -143,8 +144,10 @@ public class BasicAuthPlugin extends AuthenticationPlugin implements ConfigEdita
       } else {
         request.setAttribute(AuthenticationPlugin.class.getName(), zkAuthentication.getPromptHeaders());
         filterChain.doFilter(request, response);
+        return true;
       }
     }
+    return false;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/core/src/java/org/apache/solr/security/DelegationTokenKerberosFilter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/DelegationTokenKerberosFilter.java b/solr/core/src/java/org/apache/solr/security/DelegationTokenKerberosFilter.java
new file mode 100644
index 0000000..7dbb1ad
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/security/DelegationTokenKerberosFilter.java
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.security;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.LinkedList;
+import java.util.List;
+
+import javax.servlet.FilterChain;
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletException;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletRequestWrapper;
+
+import org.apache.curator.RetryPolicy;
+import org.apache.curator.framework.AuthInfo;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.api.ACLProvider;
+import org.apache.curator.retry.ExponentialBackoffRetry;
+
+import org.apache.hadoop.security.authentication.server.AuthenticationHandler;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticationFilter;
+import org.apache.solr.common.cloud.SecurityAwareZkACLProvider;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.cloud.ZkACLProvider;
+import org.apache.solr.common.cloud.ZkCredentialsProvider;
+import org.apache.zookeeper.data.ACL;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DelegationTokenKerberosFilter extends DelegationTokenAuthenticationFilter {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private CuratorFramework curatorFramework;
+
+  @Override
+  public void init(FilterConfig conf) throws ServletException {
+    if (conf != null && "zookeeper".equals(conf.getInitParameter("signer.secret.provider"))) {
+      SolrZkClient zkClient =
+          (SolrZkClient)conf.getServletContext().getAttribute(KerberosPlugin.DELEGATION_TOKEN_ZK_CLIENT);
+      conf.getServletContext().setAttribute("signer.secret.provider.zookeeper.curator.client",
+          getCuratorClient(zkClient));
+    }
+    super.init(conf);
+  }
+
+  @Override
+  public void doFilter(ServletRequest request, ServletResponse response,
+      FilterChain filterChain) throws IOException, ServletException {
+    // HttpClient 4.4.x throws NPE if query string is null and parsed through URLEncodedUtils.
+    // See HTTPCLIENT-1746 and HADOOP-12767
+    HttpServletRequest httpRequest = (HttpServletRequest)request;
+    String queryString = httpRequest.getQueryString();
+    final String nonNullQueryString = queryString == null ? "" : queryString;
+    HttpServletRequest requestNonNullQueryString = new HttpServletRequestWrapper(httpRequest){
+      @Override
+      public String getQueryString() {
+        return nonNullQueryString;
+      }
+    };
+    super.doFilter(requestNonNullQueryString, response, filterChain);
+  }
+
+  @Override
+  public void destroy() {
+    super.destroy();
+    if (curatorFramework != null) curatorFramework.close();
+    curatorFramework = null;
+  }
+
+  @Override
+  protected void initializeAuthHandler(String authHandlerClassName,
+                                       FilterConfig filterConfig) throws ServletException {
+    // set the internal authentication handler in order to record whether the request should continue
+    super.initializeAuthHandler(authHandlerClassName, filterConfig);
+    AuthenticationHandler authHandler = getAuthenticationHandler();
+    super.initializeAuthHandler(KerberosPlugin.RequestContinuesRecorderAuthenticationHandler.class.getName(), filterConfig);
+    KerberosPlugin.RequestContinuesRecorderAuthenticationHandler newAuthHandler =
+        (KerberosPlugin.RequestContinuesRecorderAuthenticationHandler)getAuthenticationHandler();
+    newAuthHandler.setAuthHandler(authHandler);
+  }
+
+  protected CuratorFramework getCuratorClient(SolrZkClient zkClient) {
+    // should we try to build a RetryPolicy off of the ZkController?
+    RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3);
+    if (zkClient == null) {
+      throw new IllegalArgumentException("zkClient required");
+    }
+    String zkHost = zkClient.getZkServerAddress();
+    String zkChroot = zkHost.substring(zkHost.indexOf("/"));
+    zkChroot = zkChroot.startsWith("/") ? zkChroot.substring(1) : zkChroot;
+    String zkNamespace = zkChroot + SecurityAwareZkACLProvider.SECURITY_ZNODE_PATH;
+    String zkConnectionString = zkHost.substring(0, zkHost.indexOf("/"));
+    SolrZkToCuratorCredentialsACLs curatorToSolrZk = new SolrZkToCuratorCredentialsACLs(zkClient);
+    final int connectionTimeoutMs = 30000; // this value is currently hard coded, see SOLR-7561.
+
+    curatorFramework = CuratorFrameworkFactory.builder()
+        .namespace(zkNamespace)
+        .connectString(zkConnectionString)
+        .retryPolicy(retryPolicy)
+        .aclProvider(curatorToSolrZk.getACLProvider())
+        .authorization(curatorToSolrZk.getAuthInfos())
+        .sessionTimeoutMs(zkClient.getZkClientTimeout())
+        .connectionTimeoutMs(connectionTimeoutMs)
+        .build();
+    curatorFramework.start();
+    return curatorFramework;
+  }
+
+  /**
+   * Convert Solr Zk Credentials/ACLs to Curator versions
+   */
+  protected static class SolrZkToCuratorCredentialsACLs {
+    private final ACLProvider aclProvider;
+    private final List<AuthInfo> authInfos;
+
+    public SolrZkToCuratorCredentialsACLs(SolrZkClient zkClient) {
+      this.aclProvider = createACLProvider(zkClient);
+      this.authInfos = createAuthInfo(zkClient);
+    }
+
+    public ACLProvider getACLProvider() { return aclProvider; }
+    public List<AuthInfo> getAuthInfos() { return authInfos; }
+
+    private ACLProvider createACLProvider(SolrZkClient zkClient) {
+      final ZkACLProvider zkACLProvider = zkClient.getZkACLProvider();
+      return new ACLProvider() {
+        @Override
+        public List<ACL> getDefaultAcl() {
+          return zkACLProvider.getACLsToAdd(null);
+        }
+
+        @Override
+        public List<ACL> getAclForPath(String path) {
+           List<ACL> acls = zkACLProvider.getACLsToAdd(path);
+           return acls;
+        }
+      };
+    }
+
+    private List<AuthInfo> createAuthInfo(SolrZkClient zkClient) {
+      List<AuthInfo> ret = new LinkedList<AuthInfo>();
+
+      // In theory the credentials to add could change here if zookeeper hasn't been initialized
+      ZkCredentialsProvider credentialsProvider =
+        zkClient.getZkClientConnectionStrategy().getZkCredentialsToAddAutomatically();
+      for (ZkCredentialsProvider.ZkCredentials zkCredentials : credentialsProvider.getCredentials()) {
+        ret.add(new AuthInfo(zkCredentials.getScheme(), zkCredentials.getAuth()));
+      }
+      return ret;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/core/src/java/org/apache/solr/security/KerberosFilter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/KerberosFilter.java b/solr/core/src/java/org/apache/solr/security/KerberosFilter.java
index ee23488..9c53050 100644
--- a/solr/core/src/java/org/apache/solr/security/KerberosFilter.java
+++ b/solr/core/src/java/org/apache/solr/security/KerberosFilter.java
@@ -26,6 +26,7 @@ import javax.servlet.ServletResponse;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
+import org.apache.hadoop.security.authentication.server.AuthenticationHandler;
 
 public class KerberosFilter extends AuthenticationFilter {
   
@@ -35,6 +36,19 @@ public class KerberosFilter extends AuthenticationFilter {
   }
 
   @Override
+  protected void initializeAuthHandler(String authHandlerClassName,
+                                       FilterConfig filterConfig) throws ServletException {
+    // set the internal authentication handler in order to record whether the request should continue
+    super.initializeAuthHandler(authHandlerClassName, filterConfig);
+    AuthenticationHandler authHandler = getAuthenticationHandler();
+    super.initializeAuthHandler(
+        KerberosPlugin.RequestContinuesRecorderAuthenticationHandler.class.getName(), filterConfig);
+    KerberosPlugin.RequestContinuesRecorderAuthenticationHandler newAuthHandler =
+        (KerberosPlugin.RequestContinuesRecorderAuthenticationHandler)getAuthenticationHandler();
+    newAuthHandler.setAuthHandler(authHandler);
+  }
+
+  @Override
   protected void doFilter(FilterChain filterChain, HttpServletRequest request,
       HttpServletResponse response) throws IOException, ServletException {
     super.doFilter(filterChain, request, response);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/core/src/java/org/apache/solr/security/KerberosPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/KerberosPlugin.java b/solr/core/src/java/org/apache/solr/security/KerberosPlugin.java
index 7a83ab5..1cd476f 100644
--- a/solr/core/src/java/org/apache/solr/security/KerberosPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/KerberosPlugin.java
@@ -16,14 +16,18 @@
  */
 package org.apache.solr.security;
 
+import java.io.IOException;
 import java.io.InputStream;
+import java.io.PrintWriter;
 import java.lang.invoke.MethodHandles;
 import java.net.MalformedURLException;
 import java.net.URL;
+import java.util.Collections;
 import java.util.Enumeration;
 import java.util.EventListener;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Properties;
 import java.util.Set;
 
 import javax.servlet.Filter;
@@ -41,12 +45,22 @@ import javax.servlet.ServletResponse;
 import javax.servlet.SessionCookieConfig;
 import javax.servlet.SessionTrackingMode;
 import javax.servlet.descriptor.JspConfigDescriptor;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.servlet.http.HttpServletResponseWrapper;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.collections.iterators.IteratorEnumeration;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.hadoop.security.authentication.server.AuthenticationHandler;
+import org.apache.hadoop.security.authentication.server.AuthenticationToken;
 import org.apache.solr.client.solrj.impl.Krb5HttpClientBuilder;
 import org.apache.solr.client.solrj.impl.SolrHttpClientBuilder;
+import org.apache.solr.cloud.ZkController;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.cloud.SecurityAwareZkACLProvider;
+import org.apache.solr.common.util.SuppressForbidden;
 import org.apache.solr.core.CoreContainer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -55,7 +69,7 @@ public class KerberosPlugin extends AuthenticationPlugin implements HttpClientBu
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   Krb5HttpClientBuilder kerberosBuilder = new Krb5HttpClientBuilder();
-  Filter kerberosFilter = new KerberosFilter();
+  Filter kerberosFilter;
   
   public static final String NAME_RULES_PARAM = "solr.kerberos.name.rules";
   public static final String COOKIE_DOMAIN_PARAM = "solr.kerberos.cookie.domain";
@@ -64,6 +78,26 @@ public class KerberosPlugin extends AuthenticationPlugin implements HttpClientBu
   public static final String KEYTAB_PARAM = "solr.kerberos.keytab";
   public static final String TOKEN_VALID_PARAM = "solr.kerberos.token.valid";
   public static final String COOKIE_PORT_AWARE_PARAM = "solr.kerberos.cookie.portaware";
+  public static final String DELEGATION_TOKEN_ENABLED = "solr.kerberos.delegation.token.enabled";
+  public static final String DELEGATION_TOKEN_KIND = "solr.kerberos.delegation.token.kind";
+  public static final String DELEGATION_TOKEN_VALIDITY = "solr.kerberos.delegation.token.validity";
+  public static final String DELEGATION_TOKEN_SECRET_PROVIDER = "solr.kerberos.delegation.token.signer.secret.provider";
+  public static final String DELEGATION_TOKEN_SECRET_PROVIDER_ZK_PATH =
+      "solr.kerberos.delegation.token.signer.secret.provider.zookeper.path";
+  public static final String DELEGATION_TOKEN_SECRET_MANAGER_ZNODE_WORKING_PATH =
+      "solr.kerberos.delegation.token.secret.manager.znode.working.path";
+  public static final String DELEGATION_TOKEN_TYPE_DEFAULT = "solr-dt";
+  
+  // filled in by Plugin/Filter
+  static final String REQUEST_CONTINUES_ATTR =
+      "org.apache.solr.security.kerberosplugin.requestcontinues";
+  static final String DELEGATION_TOKEN_ZK_CLIENT =
+      "solr.kerberos.delegation.token.zk.client";
+
+  // allows test to specify an alternate auth handler
+  @VisibleForTesting
+  public static final String AUTH_HANDLER_PARAM = "solr.kerberos.auth.handler";
+
   private final CoreContainer coreContainer;
 
   public KerberosPlugin(CoreContainer coreContainer) {
@@ -74,12 +108,47 @@ public class KerberosPlugin extends AuthenticationPlugin implements HttpClientBu
   public void init(Map<String, Object> pluginConfig) {
     try {
       Map<String, String> params = new HashMap();
-      params.put("type", "kerberos");
+      putParam(params, "type", AUTH_HANDLER_PARAM, "kerberos");
       putParam(params, "kerberos.name.rules", NAME_RULES_PARAM, "DEFAULT");
       putParam(params, "token.valid", TOKEN_VALID_PARAM, "30");
       putParam(params, "cookie.path", COOKIE_PATH_PARAM, "/");
-      putParam(params, "kerberos.principal", PRINCIPAL_PARAM, null);
-      putParam(params, "kerberos.keytab", KEYTAB_PARAM, null);
+      if ("kerberos".equals(params.get("type"))) {
+        putParam(params, "kerberos.principal", PRINCIPAL_PARAM, null);
+        putParam(params, "kerberos.keytab", KEYTAB_PARAM, null);
+      } else {
+        // allow tests which specify AUTH_HANDLER_PARAM to avoid specifying kerberos principal/keytab
+        putParamOptional(params, "kerberos.principal", PRINCIPAL_PARAM);
+        putParamOptional(params, "kerberos.keytab", KEYTAB_PARAM);
+      }
+
+      String delegationTokenStr = System.getProperty(DELEGATION_TOKEN_ENABLED, null);
+      boolean delegationTokenEnabled =
+          (delegationTokenStr == null) ? false : Boolean.parseBoolean(delegationTokenStr);
+      ZkController controller = coreContainer.getZkController();
+
+      if (delegationTokenEnabled) {
+        putParam(params, "delegation-token.token-kind", DELEGATION_TOKEN_KIND, DELEGATION_TOKEN_TYPE_DEFAULT);
+        if (coreContainer.isZooKeeperAware()) {
+          putParam(params, "signer.secret.provider", DELEGATION_TOKEN_SECRET_PROVIDER, "zookeeper");
+          if ("zookeeper".equals(params.get("signer.secret.provider"))) {
+            String zkHost = controller.getZkServerAddress();
+            putParam(params, "token.validity", DELEGATION_TOKEN_VALIDITY, "36000");
+            params.put("zk-dt-secret-manager.enable", "true");
+            // Note - Curator complains if the znodeWorkingPath starts with /
+            String chrootPath = zkHost.substring(zkHost.indexOf("/"));
+            String relativePath = chrootPath.startsWith("/") ? chrootPath.substring(1) : chrootPath;
+            putParam(params, "zk-dt-secret-manager.znodeWorkingPath",
+                DELEGATION_TOKEN_SECRET_MANAGER_ZNODE_WORKING_PATH,
+                relativePath + SecurityAwareZkACLProvider.SECURITY_ZNODE_PATH + "/zkdtsm");
+            putParam(params, "signer.secret.provider.zookeeper.path",
+                DELEGATION_TOKEN_SECRET_PROVIDER_ZK_PATH, "/token");
+            // ensure krb5 is setup properly before running curator
+            getHttpClientBuilder(SolrHttpClientBuilder.create());
+          }
+        } else {
+          log.info("CoreContainer is not ZooKeeperAware, not setting ZK-related delegation token properties");
+        }
+      }
 
       // Special handling for the "cookie.domain" based on whether port should be
       // appended to the domain. Useful for situations where multiple solr nodes are
@@ -94,16 +163,27 @@ public class KerberosPlugin extends AuthenticationPlugin implements HttpClientBu
         if (host==null) {
           throw new SolrException(ErrorCode.SERVER_ERROR, "Missing required parameter '"+COOKIE_DOMAIN_PARAM+"'.");
         }
-        int port = coreContainer.getZkController().getHostPort();
+        int port = controller.getHostPort();
         params.put("cookie.domain", host + ":" + port);
       }
-      
+
+      final ServletContext servletContext = new AttributeOnlyServletContext();
+      if (delegationTokenEnabled) {
+        kerberosFilter = new DelegationTokenKerberosFilter();
+        // pass an attribute-enabled context in order to pass the zkClient
+        // and because the filter may pass a curator instance.
+        if (controller != null) {
+          servletContext.setAttribute(DELEGATION_TOKEN_ZK_CLIENT, controller.getZkClient());
+        }
+      } else {
+        kerberosFilter = new KerberosFilter();
+      }
       log.info("Params: "+params);
 
       FilterConfig conf = new FilterConfig() {
         @Override
         public ServletContext getServletContext() {
-          return noContext;
+          return servletContext;
         }
 
         @Override
@@ -136,11 +216,43 @@ public class KerberosPlugin extends AuthenticationPlugin implements HttpClientBu
     params.put(internalParamName, value);
   }
 
+  private void putParamOptional(Map<String, String> params, String internalParamName, String externalParamName) {
+    String value = System.getProperty(externalParamName);
+    if (value!=null) {
+      params.put(internalParamName, value);
+    }
+  }
+
   @Override
-  public void doAuthenticate(ServletRequest req, ServletResponse rsp,
+  public boolean doAuthenticate(ServletRequest req, ServletResponse rsp,
       FilterChain chain) throws Exception {
     log.debug("Request to authenticate using kerberos: "+req);
-    kerberosFilter.doFilter(req, rsp, chain);    
+
+    final HttpServletResponse frsp = (HttpServletResponse)rsp;
+
+    // kerberosFilter may close the stream and write to closed streams,
+    // see HADOOP-13346.  To work around, pass a PrintWriter that ignores
+    // closes
+    HttpServletResponse rspCloseShield = new HttpServletResponseWrapper(frsp) {
+      @SuppressForbidden(reason = "Hadoop DelegationTokenAuthenticationFilter uses response writer, this" +
+          "is providing a CloseShield on top of that")
+      @Override
+      public PrintWriter getWriter() throws IOException {
+        final PrintWriter pw = new PrintWriterWrapper(frsp.getWriter()) {
+          @Override
+          public void close() {};
+        };
+        return pw;
+      }
+    };
+    kerberosFilter.doFilter(req, rspCloseShield, chain);
+    String requestContinuesAttr = (String)req.getAttribute(REQUEST_CONTINUES_ATTR);
+    if (requestContinuesAttr == null) {
+      log.warn("Could not find " + REQUEST_CONTINUES_ATTR);
+      return false;
+    } else {
+      return Boolean.parseBoolean(requestContinuesAttr);
+    }
   }
 
   @Override
@@ -153,8 +265,9 @@ public class KerberosPlugin extends AuthenticationPlugin implements HttpClientBu
     kerberosBuilder.close();
   }
 
-  protected static ServletContext noContext = new ServletContext() {
-    
+  protected static class AttributeOnlyServletContext implements ServletContext {
+    private Map<String, Object> attributes = new HashMap<String, Object>();
+
     @Override
     public void setSessionTrackingModes(Set<SessionTrackingMode> sessionTrackingModes) {}
     
@@ -162,12 +275,16 @@ public class KerberosPlugin extends AuthenticationPlugin implements HttpClientBu
     public boolean setInitParameter(String name, String value) {
       return false;
     }
-    
+
     @Override
-    public void setAttribute(String name, Object object) {}
-    
+    public void setAttribute(String name, Object object) {
+      attributes.put(name, object);
+    }
+
     @Override
-    public void removeAttribute(String name) {}
+    public void removeAttribute(String name) {
+      attributes.remove(name);
+    }
     
     @Override
     public void log(String message, Throwable throwable) {}
@@ -327,15 +444,15 @@ public class KerberosPlugin extends AuthenticationPlugin implements HttpClientBu
     public ClassLoader getClassLoader() {
       return null;
     }
-    
+
     @Override
     public Enumeration<String> getAttributeNames() {
-      return null;
+      return Collections.enumeration(attributes.keySet());
     }
-    
+
     @Override
     public Object getAttribute(String name) {
-      return null;
+      return attributes.get(name);
     }
     
     @Override
@@ -395,4 +512,44 @@ public class KerberosPlugin extends AuthenticationPlugin implements HttpClientBu
       return null;
     }
   };
+
+  /*
+   * {@link AuthenticationHandler} that delegates to another {@link AuthenticationHandler}
+   * and records the response of managementOperation (which indicates whether the request
+   * should continue or not).
+   */
+  public static class RequestContinuesRecorderAuthenticationHandler implements AuthenticationHandler {
+    private AuthenticationHandler authHandler;
+
+    public void setAuthHandler(AuthenticationHandler authHandler) {
+      this.authHandler = authHandler;
+    }
+
+    public String getType() {
+      return authHandler.getType();
+    }
+
+    public void init(Properties config) throws ServletException {
+      // authHandler has already been init'ed, nothing to do here
+    }
+
+    public void destroy() {
+      authHandler.destroy();
+    }
+
+    public boolean managementOperation(AuthenticationToken token,
+                                       HttpServletRequest request,
+                                       HttpServletResponse response)
+        throws IOException, AuthenticationException {
+      boolean result = authHandler.managementOperation(token, request, response);
+      request.setAttribute(KerberosPlugin.REQUEST_CONTINUES_ATTR, new Boolean(result).toString());
+      return result;
+    }
+
+
+    public AuthenticationToken authenticate(HttpServletRequest request, HttpServletResponse response)
+        throws IOException, AuthenticationException {
+      return authHandler.authenticate(request, response);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java b/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
index fb3665c..e5d2653 100644
--- a/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
@@ -89,12 +89,12 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
 
   @SuppressForbidden(reason = "Needs currentTimeMillis to compare against time in header")
   @Override
-  public void doAuthenticate(ServletRequest request, ServletResponse response, FilterChain filterChain) throws Exception {
+  public boolean doAuthenticate(ServletRequest request, ServletResponse response, FilterChain filterChain) throws Exception {
 
     String requestURI = ((HttpServletRequest) request).getRequestURI();
     if (requestURI.endsWith(PATH)) {
       filterChain.doFilter(request, response);
-      return;
+      return true;
     }
     long receivedTime = System.currentTimeMillis();
     String header = ((HttpServletRequest) request).getHeader(HEADER);
@@ -102,14 +102,14 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
       //this must not happen
       log.error("No SolrAuth header present");
       filterChain.doFilter(request, response);
-      return;
+      return true;
     }
 
     List<String> authInfo = StrUtils.splitWS(header, false);
     if (authInfo.size() < 2) {
       log.error("Invalid SolrAuth Header {}", header);
       filterChain.doFilter(request, response);
-      return;
+      return true;
     }
 
     String nodeName = authInfo.get(0);
@@ -119,12 +119,12 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
     if (decipher == null) {
       log.error("Could not decipher a header {} . No principal set", header);
       filterChain.doFilter(request, response);
-      return;
+      return true;
     }
     if ((receivedTime - decipher.timestamp) > MAX_VALIDITY) {
       log.error("Invalid key request timestamp: {} , received timestamp: {} , TTL: {}", decipher.timestamp, receivedTime, MAX_VALIDITY);
         filterChain.doFilter(request, response);
-        return;
+        return true;
     }
 
     final Principal principal = "$".equals(decipher.userName) ?
@@ -132,6 +132,7 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
         new BasicUserPrincipal(decipher.userName);
 
     filterChain.doFilter(getWrapper((HttpServletRequest) request, principal), response);
+    return true;
   }
 
   private static HttpServletRequestWrapper getWrapper(final HttpServletRequest request, final Principal principal) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/core/src/java/org/apache/solr/security/PrintWriterWrapper.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/PrintWriterWrapper.java b/solr/core/src/java/org/apache/solr/security/PrintWriterWrapper.java
new file mode 100644
index 0000000..a4e47b5
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/security/PrintWriterWrapper.java
@@ -0,0 +1,215 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.security;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.Locale;
+
+import org.apache.commons.lang.NotImplementedException;
+
+/**
+ * Wrapper for PrintWriter that delegates to constructor arg
+ */
+public class PrintWriterWrapper extends PrintWriter {
+  private PrintWriter printWriter;
+
+  public PrintWriterWrapper(PrintWriter printWriter) {
+    super(new StringWriter());
+    this.printWriter = printWriter;
+  }
+
+  @Override
+  public PrintWriter append(char c) {
+    return printWriter.append(c);
+  }
+
+  @Override
+  public PrintWriter append(CharSequence csq) {
+    return printWriter.append(csq);
+  }
+
+  @Override
+  public PrintWriter append(CharSequence csq, int start, int end) {
+    return printWriter.append(csq, start, end);
+  }
+
+  @Override
+  public boolean checkError() {
+    return printWriter.checkError();
+  }
+
+  @Override
+  protected void clearError() {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public void close() {
+    printWriter.close();
+  }
+
+  @Override
+  public void flush() {
+    printWriter.flush();
+  }
+
+  @Override
+  public PrintWriter format(Locale l, String format, Object... args) {
+    return printWriter.format(l, format, args);
+  }
+
+  @Override
+  public PrintWriter format(String format, Object... args) {
+    throw new NotImplementedException("Forbidden API");
+  }
+
+  @Override
+  public void print(boolean b) {
+    printWriter.print(b);
+  }
+
+  @Override
+  public void print(char c) {
+    printWriter.print(c);
+  }
+
+  @Override
+  public void print(char[] s) {
+    printWriter.print(s);
+  }
+
+  @Override
+  public void print(double d) {
+    printWriter.print(d);
+  }
+
+  @Override
+  public void print(float f) {
+    printWriter.print(f);
+  }
+
+  @Override
+  public void print(int i) {
+    printWriter.print(i);
+  }
+
+  @Override
+  public void print(long l) {
+    printWriter.print(l);
+  }
+
+  @Override
+  public void print(Object obj) {
+    printWriter.print(obj);
+  }
+
+  @Override
+  public void print(String s) {
+    printWriter.print(s);
+  }
+
+  @Override
+  public PrintWriter printf(Locale l, String format, Object... args) {
+    return printWriter.printf(l, format, args);
+  }
+
+  @Override
+  public PrintWriter printf(String format, Object... args) {
+    throw new NotImplementedException("Forbidden API");
+  }
+
+  @Override
+  public void println() {
+    printWriter.println();
+  }
+
+  @Override
+  public void println(boolean x) {
+    printWriter.println(x);
+  }
+
+  @Override
+  public void println(char x) {
+    printWriter.println(x);
+  }
+
+  @Override
+  public void println(char[] x) {
+    printWriter.println(x);
+  }
+
+  @Override
+  public void println(double x) {
+    printWriter.println(x);
+  }
+
+  @Override
+  public void println(float x) {
+    printWriter.println(x);
+  }
+
+  @Override
+  public void println(int x) {
+    printWriter.println(x);
+  }
+
+  @Override
+  public void println(long x) {
+    printWriter.println(x);
+  }
+
+  @Override
+  public void println(Object x) {
+    printWriter.println(x);
+  }
+
+  @Override
+  public void println(String x) {
+    printWriter.println(x);
+  }
+
+  @Override
+  protected void setError() {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public void write(char[] buf) {
+    printWriter.write(buf);
+  }
+
+  @Override
+  public void write(char[] buf, int off, int len) {
+    printWriter.write(buf, off, len);
+  }
+
+  @Override
+  public void write(int c) {
+    printWriter.write(c);
+  }
+
+  @Override
+  public void write(String s) {
+    printWriter.write(s);
+  }
+
+  @Override
+  public void write(String s, int off, int len) {
+    printWriter.write(s, off, len);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
index 2d08935..4a680e5 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
@@ -296,6 +296,7 @@ public class SolrDispatchFilter extends BaseSolrFilter {
   }
 
   private boolean authenticateRequest(ServletRequest request, ServletResponse response, final AtomicReference<ServletRequest> wrappedRequest) throws IOException {
+    boolean requestContinues = false;
     final AtomicBoolean isAuthenticated = new AtomicBoolean(false);
     AuthenticationPlugin authenticationPlugin = cores.getAuthenticationPlugin();
     if (authenticationPlugin == null) {
@@ -308,7 +309,7 @@ public class SolrDispatchFilter extends BaseSolrFilter {
       try {
         log.debug("Request to authenticate: {}, domain: {}, port: {}", request, request.getLocalName(), request.getLocalPort());
         // upon successful authentication, this should call the chain's next filter.
-        authenticationPlugin.doAuthenticate(request, response, new FilterChain() {
+        requestContinues = authenticationPlugin.doAuthenticate(request, response, new FilterChain() {
           public void doFilter(ServletRequest req, ServletResponse rsp) throws IOException, ServletException {
             isAuthenticated.set(true);
             wrappedRequest.set(req);
@@ -319,8 +320,13 @@ public class SolrDispatchFilter extends BaseSolrFilter {
         throw new SolrException(ErrorCode.SERVER_ERROR, "Error during request authentication, ", e);
       }
     }
-    // failed authentication?
-    if (!isAuthenticated.get()) {
+    // requestContinues is an optional short circuit, thus we still need to check isAuthenticated.
+    // This is because the AuthenticationPlugin doesn't always have enough information to determine if
+    // it should short circuit, e.g. the Kerberos Authentication Filter will send an error and not
+    // call later filters in chain, but doesn't throw an exception.  We could force each Plugin
+    // to implement isAuthenticated to simplify the check here, but that just moves the complexity to
+    // multiple code paths.
+    if (!requestContinues || !isAuthenticated.get()) {
       response.flushBuffer();
       return false;
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/core/src/test/org/apache/solr/cloud/KerberosTestServices.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/KerberosTestServices.java b/solr/core/src/test/org/apache/solr/cloud/KerberosTestServices.java
new file mode 100644
index 0000000..eb27218
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/KerberosTestServices.java
@@ -0,0 +1,228 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.cloud;
+
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.auth.login.Configuration;
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Properties;
+
+import com.carrotsearch.ant.tasks.junit4.dependencies.com.google.common.base.Preconditions;
+import org.apache.hadoop.minikdc.MiniKdc;
+import org.apache.solr.client.solrj.impl.Krb5HttpClientBuilder;
+
+public class KerberosTestServices {
+
+  private MiniKdc kdc;
+  private JaasConfiguration jaasConfiguration;
+  private Configuration savedConfig;
+  private Locale savedLocale;
+
+  private KerberosTestServices(MiniKdc kdc,
+                               JaasConfiguration jaasConfiguration,
+                               Configuration savedConfig,
+                               Locale savedLocale) {
+    this.kdc = kdc;
+    this.jaasConfiguration = jaasConfiguration;
+    this.savedConfig = savedConfig;
+    this.savedLocale = savedLocale;
+  }
+
+  public MiniKdc getKdc() {
+    return kdc;
+  }
+
+  public void start() throws Exception {
+    if (kdc != null) kdc.start();
+    Configuration.setConfiguration(jaasConfiguration);
+    Krb5HttpClientBuilder.regenerateJaasConfiguration();
+    if (brokenLanguagesWithMiniKdc.contains(Locale.getDefault().getLanguage())) {
+      Locale.setDefault(Locale.US);
+    }
+  }
+
+  public void stop() {
+    if (kdc != null) kdc.stop();
+    Configuration.setConfiguration(savedConfig);
+    Krb5HttpClientBuilder.regenerateJaasConfiguration();
+    Locale.setDefault(savedLocale);
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  /**
+   * Returns a MiniKdc that can be used for creating kerberos principals
+   * and keytabs.  Caller is responsible for starting/stopping the kdc.
+   */
+  private static MiniKdc getKdc(File workDir) throws Exception {
+    Properties conf = MiniKdc.createConf();
+    return new MiniKdc(conf, workDir);
+  }
+
+  /**
+   * Programmatic version of a jaas.conf file suitable for connecting
+   * to a SASL-configured zookeeper.
+   */
+  private static class JaasConfiguration extends Configuration {
+
+    private static AppConfigurationEntry[] clientEntry;
+    private static AppConfigurationEntry[] serverEntry;
+    private String clientAppName = "Client", serverAppName = "Server";
+
+    /**
+     * Add an entry to the jaas configuration with the passed in name,
+     * principal, and keytab. The other necessary options will be set for you.
+     *
+     * @param clientPrincipal The principal of the client
+     * @param clientKeytab The location of the keytab with the clientPrincipal
+     * @param serverPrincipal The principal of the server
+     * @param serverKeytab The location of the keytab with the serverPrincipal
+     */
+    public JaasConfiguration(String clientPrincipal, File clientKeytab,
+                             String serverPrincipal, File serverKeytab) {
+      Map<String, String> clientOptions = new HashMap();
+      clientOptions.put("principal", clientPrincipal);
+      clientOptions.put("keyTab", clientKeytab.getAbsolutePath());
+      clientOptions.put("useKeyTab", "true");
+      clientOptions.put("storeKey", "true");
+      clientOptions.put("useTicketCache", "false");
+      clientOptions.put("refreshKrb5Config", "true");
+      String jaasProp = System.getProperty("solr.jaas.debug");
+      if (jaasProp != null && "true".equalsIgnoreCase(jaasProp)) {
+        clientOptions.put("debug", "true");
+      }
+      clientEntry = new AppConfigurationEntry[]{
+          new AppConfigurationEntry(getKrb5LoginModuleName(),
+              AppConfigurationEntry.LoginModuleControlFlag.REQUIRED,
+              clientOptions)};
+      if(serverPrincipal!=null && serverKeytab!=null) {
+        Map<String, String> serverOptions = new HashMap(clientOptions);
+        serverOptions.put("principal", serverPrincipal);
+        serverOptions.put("keytab", serverKeytab.getAbsolutePath());
+        serverEntry =  new AppConfigurationEntry[]{
+            new AppConfigurationEntry(getKrb5LoginModuleName(),
+                AppConfigurationEntry.LoginModuleControlFlag.REQUIRED,
+                serverOptions)};
+      }
+    }
+
+    /**
+     * Add an entry to the jaas configuration with the passed in principal and keytab,
+     * along with the app name.
+     *
+     * @param principal The principal
+     * @param keytab The keytab containing credentials for the principal
+     * @param appName The app name of the configuration
+     */
+    public JaasConfiguration(String principal, File keytab, String appName) {
+      this(principal, keytab, null, null);
+      clientAppName = appName;
+      serverAppName = null;
+    }
+
+    @Override
+    public AppConfigurationEntry[] getAppConfigurationEntry(String name) {
+      if (name.equals(clientAppName)) {
+        return clientEntry;
+      } else if (name.equals(serverAppName)) {
+        return serverEntry;
+      }
+      return null;
+    }
+
+    private String getKrb5LoginModuleName() {
+      String krb5LoginModuleName;
+      if (System.getProperty("java.vendor").contains("IBM")) {
+        krb5LoginModuleName = "com.ibm.security.auth.module.Krb5LoginModule";
+      } else {
+        krb5LoginModuleName = "com.sun.security.auth.module.Krb5LoginModule";
+      }
+      return krb5LoginModuleName;
+    }
+  }
+
+  /**
+   *  These Locales don't generate dates that are compatibile with Hadoop MiniKdc.
+   */
+  private final static List<String> brokenLanguagesWithMiniKdc =
+      Arrays.asList(
+          new Locale("th").getLanguage(),
+          new Locale("ja").getLanguage(),
+          new Locale("hi").getLanguage()
+      );
+
+  public static class Builder {
+    private File kdcWorkDir;
+    private String clientPrincipal;
+    private File clientKeytab;
+    private String serverPrincipal;
+    private File serverKeytab;
+    private String appName;
+    private Locale savedLocale;
+
+    public Builder() {
+      savedLocale = Locale.getDefault();
+    }
+
+    public Builder withKdc(File kdcWorkDir) {
+      this.kdcWorkDir = kdcWorkDir;
+      return this;
+    }
+
+    public Builder withJaasConfiguration(String clientPrincipal, File clientKeytab,
+                                         String serverPrincipal, File serverKeytab) {
+      Preconditions.checkNotNull(clientPrincipal);
+      Preconditions.checkNotNull(clientKeytab);
+      this.clientPrincipal = clientPrincipal;
+      this.clientKeytab = clientKeytab;
+      this.serverPrincipal = serverPrincipal;
+      this.serverKeytab = serverKeytab;
+      this.appName = null;
+      return this;
+    }
+
+    public Builder withJaasConfiguration(String principal, File keytab, String appName) {
+      Preconditions.checkNotNull(principal);
+      Preconditions.checkNotNull(keytab);
+      this.clientPrincipal = principal;
+      this.clientKeytab = keytab;
+      this.serverPrincipal = null;
+      this.serverKeytab = null;
+      this.appName = appName;
+      return this;
+    }
+
+    public KerberosTestServices build() throws Exception {
+      final MiniKdc kdc = kdcWorkDir != null ? getKdc(kdcWorkDir) : null;
+      final Configuration oldConfig = clientPrincipal != null ? Configuration.getConfiguration() : null;
+      JaasConfiguration jaasConfiguration = null;
+      if (clientPrincipal != null) {
+        jaasConfiguration = (appName == null) ?
+            new JaasConfiguration(clientPrincipal, clientKeytab, serverPrincipal, serverKeytab) :
+            new JaasConfiguration(clientPrincipal, clientKeytab, appName);
+      }
+      return new KerberosTestServices(kdc, jaasConfiguration, oldConfig, savedLocale);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/core/src/test/org/apache/solr/cloud/KerberosTestUtil.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/KerberosTestUtil.java b/solr/core/src/test/org/apache/solr/cloud/KerberosTestUtil.java
deleted file mode 100644
index 7f544ef..0000000
--- a/solr/core/src/test/org/apache/solr/cloud/KerberosTestUtil.java
+++ /dev/null
@@ -1,147 +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.cloud;
-
-import java.io.File;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Properties;
-
-import javax.security.auth.login.AppConfigurationEntry;
-import javax.security.auth.login.Configuration;
-
-import org.apache.hadoop.minikdc.MiniKdc;
-
-public class KerberosTestUtil {
-
-  /**
-   * Returns a MiniKdc that can be used for creating kerberos principals
-   * and keytabs.  Caller is responsible for starting/stopping the kdc.
-   */
-  public static MiniKdc getKdc(File workDir) throws Exception {
-    Properties conf = MiniKdc.createConf();
-    return new MiniKdc(conf, workDir);
-  }
-
-  /**
-   * Programmatic version of a jaas.conf file suitable for connecting
-   * to a SASL-configured zookeeper.
-   */
-  public static class JaasConfiguration extends Configuration {
-
-    private static AppConfigurationEntry[] clientEntry;
-    private static AppConfigurationEntry[] serverEntry;
-    private String clientAppName = "Client", serverAppName = "Server";
-
-    /**
-     * Add an entry to the jaas configuration with the passed in name,
-     * principal, and keytab. The other necessary options will be set for you.
-     *
-     * @param clientPrincipal The principal of the client
-     * @param clientKeytab The location of the keytab with the clientPrincipal
-     * @param serverPrincipal The principal of the server
-     * @param serverKeytab The location of the keytab with the serverPrincipal
-     */
-    public JaasConfiguration(String clientPrincipal, File clientKeytab,
-        String serverPrincipal, File serverKeytab) {
-      Map<String, String> clientOptions = new HashMap();
-      clientOptions.put("principal", clientPrincipal);
-      clientOptions.put("keyTab", clientKeytab.getAbsolutePath());
-      clientOptions.put("useKeyTab", "true");
-      clientOptions.put("storeKey", "true");
-      clientOptions.put("useTicketCache", "false");
-      clientOptions.put("refreshKrb5Config", "true");
-      String jaasProp = System.getProperty("solr.jaas.debug");
-      if (jaasProp != null && "true".equalsIgnoreCase(jaasProp)) {
-        clientOptions.put("debug", "true");
-      }
-      clientEntry = new AppConfigurationEntry[]{
-        new AppConfigurationEntry(getKrb5LoginModuleName(),
-        AppConfigurationEntry.LoginModuleControlFlag.REQUIRED,
-        clientOptions)};
-      if(serverPrincipal!=null && serverKeytab!=null) {
-        Map<String, String> serverOptions = new HashMap(clientOptions);
-        serverOptions.put("principal", serverPrincipal);
-        serverOptions.put("keytab", serverKeytab.getAbsolutePath());
-        serverEntry =  new AppConfigurationEntry[]{
-            new AppConfigurationEntry(getKrb5LoginModuleName(),
-                AppConfigurationEntry.LoginModuleControlFlag.REQUIRED,
-                serverOptions)};
-      }
-    }
-
-    /**
-     * Add an entry to the jaas configuration with the passed in principal and keytab, 
-     * along with the app name.
-     * 
-     * @param principal The principal
-     * @param keytab The keytab containing credentials for the principal
-     * @param appName The app name of the configuration
-     */
-    public JaasConfiguration(String principal, File keytab, String appName) {
-      this(principal, keytab, null, null);
-      clientAppName = appName;
-      serverAppName = null;
-    }
-
-    @Override
-    public AppConfigurationEntry[] getAppConfigurationEntry(String name) {
-      if (name.equals(clientAppName)) {
-        return clientEntry;
-      } else if (name.equals(serverAppName)) {
-        return serverEntry;
-      }
-      return null;
-    }
-
-    private String getKrb5LoginModuleName() {
-      String krb5LoginModuleName;
-      if (System.getProperty("java.vendor").contains("IBM")) {
-        krb5LoginModuleName = "com.ibm.security.auth.module.Krb5LoginModule";
-      } else {
-        krb5LoginModuleName = "com.sun.security.auth.module.Krb5LoginModule";
-      }
-      return krb5LoginModuleName;
-    }
-  }
-
-  /**
-   *  These Locales don't generate dates that are compatibile with Hadoop MiniKdc.
-   */
-  private final static List<String> brokenLanguagesWithMiniKdc =
-      Arrays.asList(
-          new Locale("th").getLanguage(), 
-          new Locale("ja").getLanguage(), 
-          new Locale("hi").getLanguage()
-          );
-  /** 
-   *returns the currently set locale, and overrides it with {@link Locale#US} if it's 
-   * currently something MiniKdc can not handle
-   *
-   * @see Locale#setDefault
-   */
-  public static final Locale overrideLocaleIfNotSpportedByMiniKdc() {
-    Locale old = Locale.getDefault();
-    if (brokenLanguagesWithMiniKdc.contains(Locale.getDefault().getLanguage())) {
-      Locale.setDefault(Locale.US);
-    }
-    return old;
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/core/src/test/org/apache/solr/cloud/OutOfBoxZkACLAndCredentialsProvidersTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/OutOfBoxZkACLAndCredentialsProvidersTest.java b/solr/core/src/test/org/apache/solr/cloud/OutOfBoxZkACLAndCredentialsProvidersTest.java
index 51ad523..0884576 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OutOfBoxZkACLAndCredentialsProvidersTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OutOfBoxZkACLAndCredentialsProvidersTest.java
@@ -23,6 +23,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.cloud.SecurityAwareZkACLProvider;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.ZooDefs;
@@ -77,6 +78,7 @@ public class OutOfBoxZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
     zkClient.makePath("/protectedMakePathNode", "content".getBytes(DATA_ENCODING), CreateMode.PERSISTENT, false);
     zkClient.create("/unprotectedCreateNode", "content".getBytes(DATA_ENCODING), CreateMode.PERSISTENT, false);
     zkClient.makePath("/unprotectedMakePathNode", "content".getBytes(DATA_ENCODING), CreateMode.PERSISTENT, false);
+    zkClient.create(SecurityAwareZkACLProvider.SECURITY_ZNODE_PATH, "content".getBytes(DATA_ENCODING), CreateMode.PERSISTENT, false);
     zkClient.close();
 
     log.info("####SETUP_END " + getTestName());
@@ -93,7 +95,9 @@ public class OutOfBoxZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
   public void testOutOfBoxSolrZkClient() throws Exception {
     SolrZkClient zkClient = new SolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT);
     try {
-      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient, true, true, true, true, true);
+      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient,
+          true, true, true, true, true,
+          true, true, true, true, true);
     } finally {
       zkClient.close();
     }
@@ -110,6 +114,7 @@ public class OutOfBoxZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
       assertTrue(verifiedList.contains("/solr/unprotectedMakePathNode"));
       assertTrue(verifiedList.contains("/solr/protectedMakePathNode"));
       assertTrue(verifiedList.contains("/solr/protectedCreateNode"));
+      assertTrue(verifiedList.contains("/solr" + SecurityAwareZkACLProvider.SECURITY_ZNODE_PATH));
     } finally {
       zkClient.close();
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/core/src/test/org/apache/solr/cloud/OverriddenZkACLAndCredentialsProvidersTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/OverriddenZkACLAndCredentialsProvidersTest.java b/solr/core/src/test/org/apache/solr/cloud/OverriddenZkACLAndCredentialsProvidersTest.java
index b87ab1b..56c0df9 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OverriddenZkACLAndCredentialsProvidersTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OverriddenZkACLAndCredentialsProvidersTest.java
@@ -18,18 +18,15 @@ package org.apache.solr.cloud;
 
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.StringUtils;
-import org.apache.solr.common.cloud.DefaultZkACLProvider;
 import org.apache.solr.common.cloud.DefaultZkCredentialsProvider;
+import org.apache.solr.common.cloud.SecurityAwareZkACLProvider;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.VMParamsAllAndReadonlyDigestZkACLProvider;
 import org.apache.solr.common.cloud.VMParamsSingleSetCredentialsDigestZkCredentialsProvider;
 import org.apache.solr.common.cloud.ZkACLProvider;
 import org.apache.solr.common.cloud.ZkCredentialsProvider;
 import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.data.ACL;
-import org.apache.zookeeper.data.Id;
-import org.apache.zookeeper.server.auth.DigestAuthenticationProvider;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -40,7 +37,6 @@ import java.io.File;
 import java.io.UnsupportedEncodingException;
 import java.lang.invoke.MethodHandles;
 import java.nio.charset.Charset;
-import java.security.NoSuchAlgorithmException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
@@ -88,6 +84,7 @@ public class OverriddenZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
         "readonlyACLUsername", "readonlyACLPassword").getSolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT);
     zkClient.create("/protectedCreateNode", "content".getBytes(DATA_ENCODING), CreateMode.PERSISTENT, false);
     zkClient.makePath("/protectedMakePathNode", "content".getBytes(DATA_ENCODING), CreateMode.PERSISTENT, false);
+    zkClient.create(SecurityAwareZkACLProvider.SECURITY_ZNODE_PATH, "content".getBytes(DATA_ENCODING), CreateMode.PERSISTENT, false);
     zkClient.close();
     
     zkClient = new SolrZkClientFactoryUsingCompletelyNewProviders(null, null, 
@@ -114,7 +111,9 @@ public class OverriddenZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
     SolrZkClient zkClient = new SolrZkClientFactoryUsingCompletelyNewProviders(null, null, 
         null, null).getSolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT);
     try {
-      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient, false, false, false, false, false);
+      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient,
+          false, false, false, false, false,
+          false, false, false, false, false);
     } finally {
       zkClient.close();
     }
@@ -125,7 +124,9 @@ public class OverriddenZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
     SolrZkClient zkClient = new SolrZkClientFactoryUsingCompletelyNewProviders("connectAndAllACLUsername", "connectAndAllACLPasswordWrong", 
         null, null).getSolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT);
     try {
-      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient, false, false, false, false, false);
+      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient,
+          false, false, false, false, false,
+          false, false, false, false, false);
     } finally {
       zkClient.close();
     }
@@ -136,7 +137,9 @@ public class OverriddenZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
     SolrZkClient zkClient = new SolrZkClientFactoryUsingCompletelyNewProviders("connectAndAllACLUsername", "connectAndAllACLPassword", 
         null, null).getSolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT);
     try {
-      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient, true, true, true, true, true);
+      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient,
+          true, true, true, true, true,
+          true, true, true, true, true);
     } finally {
       zkClient.close();
     }
@@ -147,7 +150,9 @@ public class OverriddenZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
     SolrZkClient zkClient = new SolrZkClientFactoryUsingCompletelyNewProviders("readonlyACLUsername", "readonlyACLPassword",
         null, null).getSolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT);
     try {
-      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient, true, true, false, false, false);
+      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient,
+          true, true, false, false, false,
+          false, false, false, false, false);
     } finally {
       zkClient.close();
     }
@@ -159,7 +164,9 @@ public class OverriddenZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
     
     SolrZkClient zkClient = new SolrZkClientUsingVMParamsProvidersButWithDifferentVMParamsNames(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT);
     try {
-      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient, false, false, false, false, false);
+      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient,
+          false, false, false, false, false,
+          false, false, false, false, false);
     } finally {
       zkClient.close();
     }
@@ -171,7 +178,9 @@ public class OverriddenZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
     
     SolrZkClient zkClient = new SolrZkClientUsingVMParamsProvidersButWithDifferentVMParamsNames(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT);
     try {
-      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient, false, false, false, false, false);
+      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient,
+          false, false, false, false, false,
+          false, false, false, false, false);
     } finally {
       zkClient.close();
     }
@@ -183,7 +192,9 @@ public class OverriddenZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
     
     SolrZkClient zkClient = new SolrZkClientUsingVMParamsProvidersButWithDifferentVMParamsNames(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT);
     try {
-      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient, true, true, true, true, true);
+      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient,
+          true, true, true, true, true,
+          true, true, true, true, true);
     } finally {
       zkClient.close();
     }
@@ -195,7 +206,9 @@ public class OverriddenZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
     
     SolrZkClient zkClient = new SolrZkClientUsingVMParamsProvidersButWithDifferentVMParamsNames(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT);
     try {
-      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient, true, true, false, false, false);
+      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient,
+          true, true, false, false, false,
+          false, false, false, false, false);
     } finally {
       zkClient.close();
     }
@@ -240,28 +253,18 @@ public class OverriddenZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
 
         @Override
         public ZkACLProvider createZkACLProvider() {
-          return new DefaultZkACLProvider() {
+          return new VMParamsAllAndReadonlyDigestZkACLProvider() {
             @Override
-            protected List<ACL> createGlobalACLsToAdd() {
-              try {
-                List<ACL> result = new ArrayList<ACL>();
-            
-                if (!StringUtils.isEmpty(digestUsername) && !StringUtils.isEmpty(digestPassword)) {
-                  result.add(new ACL(ZooDefs.Perms.ALL, new Id("digest", DigestAuthenticationProvider.generateDigest(digestUsername + ":" + digestPassword))));
-                }
-            
-                if (!StringUtils.isEmpty(digestReadonlyUsername) && !StringUtils.isEmpty(digestReadonlyPassword)) {
-                  result.add(new ACL(ZooDefs.Perms.READ, new Id("digest", DigestAuthenticationProvider.generateDigest(digestReadonlyUsername + ":" + digestReadonlyPassword))));
-                }
-                
-                if (result.isEmpty()) {
-                  result = ZooDefs.Ids.OPEN_ACL_UNSAFE;
-                }
-                
-                return result;
-              } catch (NoSuchAlgorithmException e) {
-                throw new RuntimeException(e);
-              }
+            protected List<ACL> createNonSecurityACLsToAdd() {
+              return createACLsToAdd(true, digestUsername, digestPassword, digestReadonlyUsername, digestReadonlyPassword);
+            }
+
+            /**
+             * @return Set of ACLs to return security-related znodes
+             */
+            @Override
+            protected List<ACL> createSecurityACLsToAdd() {
+              return createACLsToAdd(false, digestUsername, digestPassword, digestReadonlyUsername, digestReadonlyPassword);
             }
           };
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/core/src/test/org/apache/solr/cloud/SaslZkACLProviderTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/SaslZkACLProviderTest.java b/solr/core/src/test/org/apache/solr/cloud/SaslZkACLProviderTest.java
index 9381c03..16b67a3 100644
--- a/solr/core/src/test/org/apache/solr/cloud/SaslZkACLProviderTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/SaslZkACLProviderTest.java
@@ -20,15 +20,12 @@ import java.io.File;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.nio.charset.Charset;
-import java.util.Locale;
 
-import javax.security.auth.login.Configuration;
-
-import org.apache.hadoop.minikdc.MiniKdc;
 import org.apache.lucene.util.Constants;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.cloud.DefaultZkACLProvider;
 import org.apache.solr.common.cloud.SaslZkACLProvider;
+import org.apache.solr.common.cloud.SecurityAwareZkACLProvider;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkACLProvider;
 import org.apache.solr.util.BadZookeeperThreadsFilter;
@@ -50,8 +47,6 @@ public class SaslZkACLProviderTest extends SolrTestCaseJ4 {
 
   private static final Charset DATA_ENCODING = Charset.forName("UTF-8");
 
-  protected Locale savedLocale = null;
-
   protected ZkTestServer zkServer;
 
   @BeforeClass
@@ -71,7 +66,6 @@ public class SaslZkACLProviderTest extends SolrTestCaseJ4 {
   @Override
   public void setUp() throws Exception {
     super.setUp();
-    savedLocale = KerberosTestUtil.overrideLocaleIfNotSpportedByMiniKdc();
     log.info("####SETUP_START " + getTestName());
     createTempDir();
 
@@ -99,6 +93,7 @@ public class SaslZkACLProviderTest extends SolrTestCaseJ4 {
     try {
       zkClient.create("/protectedCreateNode", "content".getBytes(DATA_ENCODING), CreateMode.PERSISTENT, false);
       zkClient.makePath("/protectedMakePathNode", "content".getBytes(DATA_ENCODING), CreateMode.PERSISTENT, false);
+      zkClient.create(SecurityAwareZkACLProvider.SECURITY_ZNODE_PATH, "content".getBytes(DATA_ENCODING), CreateMode.PERSISTENT, false);
     } finally {
       zkClient.close();
     }
@@ -115,7 +110,6 @@ public class SaslZkACLProviderTest extends SolrTestCaseJ4 {
   @Override
   public void tearDown() throws Exception {
     zkServer.shutdown();
-    Locale.setDefault(savedLocale);
     super.tearDown();
   }
 
@@ -124,7 +118,9 @@ public class SaslZkACLProviderTest extends SolrTestCaseJ4 {
     // Test with Sasl enabled
     SolrZkClient zkClient = new SolrZkClientWithACLs(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT);
     try {
-      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient, true, true, true, true, true);
+      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient,
+          true, true, true, true, true,
+          true, true, true, true, true);
      } finally {
       zkClient.close();
     }
@@ -134,7 +130,9 @@ public class SaslZkACLProviderTest extends SolrTestCaseJ4 {
     System.setProperty("zookeeper.sasl.client", "false");
     zkClient = new SolrZkClientNoACLs(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT);
     try {
-      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient, true, true, false, false, false);
+      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient,
+          true, true, false, false, false,
+          false, false, false, false, false);
     } finally {
       zkClient.close();
       System.clearProperty("zookeeper.sasl.client");
@@ -176,8 +174,7 @@ public class SaslZkACLProviderTest extends SolrTestCaseJ4 {
    */
   public static class SaslZkTestServer extends ZkTestServer {
     private String kdcDir;
-    private MiniKdc kdc;
-    private Configuration conf;
+    private KerberosTestServices kerberosTestServices;
 
     public SaslZkTestServer(String zkDir, String kdcDir) {
       super(zkDir);
@@ -187,13 +184,11 @@ public class SaslZkACLProviderTest extends SolrTestCaseJ4 {
     public SaslZkTestServer(String zkDir, int port, String kdcDir) {
       super(zkDir, port);
       this.kdcDir = kdcDir;
-      conf = Configuration.getConfiguration();
     }
 
     @Override
     public void run() throws InterruptedException {
       try {
-        kdc = KerberosTestUtil.getKdc(new File(kdcDir));
         // Don't require that credentials match the entire principal string, e.g.
         // can match "solr" rather than "solr/host@DOMAIN"
         System.setProperty("zookeeper.kerberos.removeRealmFromPrincipal", "true");
@@ -202,12 +197,13 @@ public class SaslZkACLProviderTest extends SolrTestCaseJ4 {
         String zkClientPrincipal = "solr";
         String zkServerPrincipal = "zookeeper/127.0.0.1";
 
-        kdc.start();
-        // Create ZK client and server principals and load them into the Configuration
-        kdc.createPrincipal(keytabFile, zkClientPrincipal, zkServerPrincipal);
-        KerberosTestUtil.JaasConfiguration jaas = new KerberosTestUtil.JaasConfiguration(
-        zkClientPrincipal, keytabFile, zkServerPrincipal, keytabFile);
-        Configuration.setConfiguration(jaas);
+        kerberosTestServices = KerberosTestServices.builder()
+            .withKdc(new File(kdcDir))
+            .withJaasConfiguration(zkClientPrincipal, keytabFile, zkServerPrincipal, keytabFile)
+            .build();
+        kerberosTestServices.start();
+
+        kerberosTestServices.getKdc().createPrincipal(keytabFile, zkClientPrincipal, zkServerPrincipal);
       } catch (Exception ex) {
         throw new RuntimeException(ex);
       }
@@ -220,8 +216,7 @@ public class SaslZkACLProviderTest extends SolrTestCaseJ4 {
       System.clearProperty("zookeeper.authProvider.1");
       System.clearProperty("zookeeper.kerberos.removeRealmFromPrincipal");
       System.clearProperty("zookeeper.kerberos.removeHostFromPrincipal");
-      Configuration.setConfiguration(conf);
-      kdc.stop();
+      kerberosTestServices.stop();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/core/src/test/org/apache/solr/cloud/TestAuthenticationFramework.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestAuthenticationFramework.java b/solr/core/src/test/org/apache/solr/cloud/TestAuthenticationFramework.java
index 08db692..6b562a1 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestAuthenticationFramework.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestAuthenticationFramework.java
@@ -235,21 +235,23 @@ public class TestAuthenticationFramework extends LuceneTestCase {
     public void init(Map<String,Object> pluginConfig) {}
 
     @Override
-    public void doAuthenticate(ServletRequest request, ServletResponse response, FilterChain filterChain)
+    public boolean doAuthenticate(ServletRequest request, ServletResponse response, FilterChain filterChain)
         throws Exception {
       if (expectedUsername == null) {
         filterChain.doFilter(request, response);
-        return;
+        return true;
       }
       HttpServletRequest httpRequest = (HttpServletRequest)request;
       String username = httpRequest.getHeader("username");
       String password = httpRequest.getHeader("password");
       
       log.info("Username: "+username+", password: "+password);
-      if(MockAuthenticationPlugin.expectedUsername.equals(username) && MockAuthenticationPlugin.expectedPassword.equals(password))      
+      if(MockAuthenticationPlugin.expectedUsername.equals(username) && MockAuthenticationPlugin.expectedPassword.equals(password)) {
         filterChain.doFilter(request, response);
-      else {
+        return true;
+      } else {
         ((HttpServletResponse)response).sendError(401, "Unauthorized request");
+        return false;
       }
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudClusterKerberos.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudClusterKerberos.java b/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudClusterKerberos.java
index 37439b0..d644967 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudClusterKerberos.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudClusterKerberos.java
@@ -16,16 +16,13 @@
  */
 package org.apache.solr.cloud;
 
-import javax.security.auth.login.Configuration;
 import java.io.File;
 import java.nio.charset.StandardCharsets;
-import java.util.Locale;
 
 import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters;
 import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
 
 import org.apache.commons.io.FileUtils;
-import org.apache.hadoop.minikdc.MiniKdc;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
 import org.apache.solr.util.BadZookeeperThreadsFilter;
@@ -52,17 +49,14 @@ import org.junit.rules.TestRule;
 @SuppressSysoutChecks(bugUrl = "Solr logs to JUL")
 public class TestMiniSolrCloudClusterKerberos extends TestMiniSolrCloudCluster {
 
-  private final Configuration originalConfig = Configuration.getConfiguration();
-
   public TestMiniSolrCloudClusterKerberos () {
     NUM_SERVERS = 5;
     NUM_SHARDS = 2;
     REPLICATION_FACTOR = 2;
   }
   
-  private MiniKdc kdc;
+  private KerberosTestServices kerberosTestServices;
 
-  private Locale savedLocale; // in case locale is broken and we need to fill in a working locale
   @Rule
   public TestRule solrTestRules = RuleChain
       .outerRule(new SystemPropertiesRestoreRule());
@@ -74,20 +68,22 @@ public class TestMiniSolrCloudClusterKerberos extends TestMiniSolrCloudCluster {
 
   @Override
   public void setUp() throws Exception {
-    savedLocale = KerberosTestUtil.overrideLocaleIfNotSpportedByMiniKdc();
     super.setUp();
     setupMiniKdc();
   }
   
   private void setupMiniKdc() throws Exception {
     String kdcDir = createTempDir()+File.separator+"minikdc";
-    kdc = KerberosTestUtil.getKdc(new File(kdcDir));
     File keytabFile = new File(kdcDir, "keytabs");
     String principal = "HTTP/127.0.0.1";
     String zkServerPrincipal = "zookeeper/127.0.0.1";
+    KerberosTestServices kerberosTestServices = KerberosTestServices.builder()
+        .withKdc(new File(kdcDir))
+        .withJaasConfiguration(principal, keytabFile, zkServerPrincipal, keytabFile)
+        .build();
 
-    kdc.start();
-    kdc.createPrincipal(keytabFile, principal, zkServerPrincipal);
+    kerberosTestServices.start();
+    kerberosTestServices.getKdc().createPrincipal(keytabFile, principal, zkServerPrincipal);
 
     String jaas = "Client {\n"
         + " com.sun.security.auth.module.Krb5LoginModule required\n"
@@ -109,10 +105,7 @@ public class TestMiniSolrCloudClusterKerberos extends TestMiniSolrCloudCluster {
         + " debug=true\n"
         + " principal=\""+zkServerPrincipal+"\";\n" 
         + "};\n";
-    
-    Configuration conf = new KerberosTestUtil.JaasConfiguration(principal, keytabFile, zkServerPrincipal, keytabFile);
-    javax.security.auth.login.Configuration.setConfiguration(conf);
-    
+
     String jaasFilePath = kdcDir+File.separator + "jaas-client.conf";
     FileUtils.write(new File(jaasFilePath), jaas, StandardCharsets.UTF_8);
     System.setProperty("java.security.auth.login.config", jaasFilePath);
@@ -156,11 +149,7 @@ public class TestMiniSolrCloudClusterKerberos extends TestMiniSolrCloudCluster {
     System.clearProperty("kerberos.principal");
     System.clearProperty("kerberos.keytab");
     System.clearProperty("authenticationPlugin");
-    Configuration.setConfiguration(this.originalConfig);
-    if (kdc != null) {
-      kdc.stop();
-    }
-    Locale.setDefault(savedLocale);
+    kerberosTestServices.stop();
     super.tearDown();
   }
 }


[07/40] lucene-solr:apiv2: remove unused import (DocCollection)

Posted by sa...@apache.org.
remove unused import (DocCollection)


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

Branch: refs/heads/apiv2
Commit: a00e34495bd546752db98c5b3c6e35d66fdf593d
Parents: 7f1db8a
Author: Nicholas Knize <nk...@gmail.com>
Authored: Fri Jul 22 11:46:25 2016 -0500
Committer: Nicholas Knize <nk...@gmail.com>
Committed: Fri Jul 22 11:46:25 2016 -0500

----------------------------------------------------------------------
 .../java/org/apache/solr/client/solrj/io/stream/TopicStream.java    | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a00e3449/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java
index c8bf610..97317a0 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java
@@ -52,7 +52,6 @@ import org.apache.solr.client.solrj.request.UpdateRequest;
 import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.cloud.ClusterState;
-import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkCoreNodeProps;


[26/40] lucene-solr:apiv2: SOLR-8596: Split UIs RAW query params only on first equal sign

Posted by sa...@apache.org.
SOLR-8596: Split UIs RAW query params only on first equal sign


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

Branch: refs/heads/apiv2
Commit: c9935b478839a602616c1c6b76e0d3bb06d8f789
Parents: bc29a3d
Author: Jan H�ydahl <ja...@apache.org>
Authored: Wed Jul 27 10:02:00 2016 +1000
Committer: Jan H�ydahl <ja...@apache.org>
Committed: Thu Jul 28 15:52:43 2016 +0200

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  1 +
 solr/webapp/web/js/angular/controllers/query.js | 10 +++++++---
 2 files changed, 8 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9935b47/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index eab3780..a728c8a 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -165,6 +165,7 @@ Bug Fixes
 
 * SOLR-9339: NPE in CloudSolrClient when the response is null (noble)
 
+* SOLR-8596: Web UI doesn't correctly generate queries which include local parameters (Alexandre Rafalovitch, janhoy)
 
 Optimizations
 ----------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9935b47/solr/webapp/web/js/angular/controllers/query.js
----------------------------------------------------------------------
diff --git a/solr/webapp/web/js/angular/controllers/query.js b/solr/webapp/web/js/angular/controllers/query.js
index 8e89778..1695647 100644
--- a/solr/webapp/web/js/angular/controllers/query.js
+++ b/solr/webapp/web/js/angular/controllers/query.js
@@ -61,9 +61,13 @@ solrAdminApp.controller('QueryController',
       if ($scope.rawParams) {
         var rawParams = $scope.rawParams.split(/[&\n]/);
         for (var i in rawParams) {
-            var param = rawParams[i];
-            var parts = param.split("=");
-            set(parts[0], parts[1]);
+          var param = rawParams[i];
+          var equalPos = param.indexOf("=");
+          if (equalPos > -1) {
+            set(param.substring(0, equalPos), param.substring(equalPos+1));
+          } else {
+            set(param, ""); // Use empty value for params without "="
+          }
         }
       }
 


[10/40] lucene-solr:apiv2: SOLR-9310: The variable name is confusing. maxInHash is not a hash

Posted by sa...@apache.org.
SOLR-9310: The variable name is confusing. maxInHash is not a hash


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

Branch: refs/heads/apiv2
Commit: 830228602df1da2ba5c7e4c2688434108414e221
Parents: 9b85f68
Author: Noble Paul <no...@gmail.com>
Authored: Mon Jul 25 12:10:49 2016 +0530
Committer: Noble Paul <no...@gmail.com>
Committed: Mon Jul 25 12:15:02 2016 +0530

----------------------------------------------------------------------
 .../java/org/apache/solr/update/IndexFingerprint.java | 14 +++++++-------
 1 file changed, 7 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/83022860/solr/core/src/java/org/apache/solr/update/IndexFingerprint.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/IndexFingerprint.java b/solr/core/src/java/org/apache/solr/update/IndexFingerprint.java
index 01cf9a6..349b703 100644
--- a/solr/core/src/java/org/apache/solr/update/IndexFingerprint.java
+++ b/solr/core/src/java/org/apache/solr/update/IndexFingerprint.java
@@ -42,7 +42,7 @@ public class IndexFingerprint {
 
   private long maxVersionSpecified;
   private long maxVersionEncountered;
-  private long maxInHash;
+  private long maxVersionsUsedInHash;
   private long versionsHash;
   private long numVersions;
   private long numDocs;
@@ -56,8 +56,8 @@ public class IndexFingerprint {
     return maxVersionEncountered;
   }
 
-  public long getMaxInHash() {
-    return maxInHash;
+  public long getMaxVersionsUsedInHash() {
+    return maxVersionsUsedInHash;
   }
 
   public long getVersionsHash() {
@@ -113,7 +113,7 @@ public class IndexFingerprint {
         long v = fv.longVal(doc);
         f.maxVersionEncountered = Math.max(v, f.maxVersionEncountered);
         if (v <= f.maxVersionSpecified) {
-          f.maxInHash = Math.max(v, f.maxInHash);
+          f.maxVersionsUsedInHash = Math.max(v, f.maxVersionsUsedInHash);
           f.versionsHash += Hash.fmix64(v);
           f.numVersions++;
         }
@@ -139,7 +139,7 @@ public class IndexFingerprint {
     }
 
     // Go by the highest version under the requested max.
-    cmp = Long.compare(f1.maxInHash, f2.maxInHash);
+    cmp = Long.compare(f1.maxVersionsUsedInHash, f2.maxVersionsUsedInHash);
     if (cmp != 0) return cmp;
 
     // go by who has the most documents in the index
@@ -158,7 +158,7 @@ public class IndexFingerprint {
     Map<String,Object> map = new LinkedHashMap<>();
     map.put("maxVersionSpecified", maxVersionSpecified);
     map.put("maxVersionEncountered", maxVersionEncountered);
-    map.put("maxInHash", maxInHash);
+    map.put("maxVersionsUsedInHash", maxVersionsUsedInHash);
     map.put("versionsHash", versionsHash);
     map.put("numVersions", numVersions);
     map.put("numDocs", numDocs);
@@ -186,7 +186,7 @@ public class IndexFingerprint {
     IndexFingerprint f = new IndexFingerprint();
     f.maxVersionSpecified = getLong(o, "maxVersionSpecified", Long.MAX_VALUE);
     f.maxVersionEncountered = getLong(o, "maxVersionEncountered", -1);
-    f.maxInHash = getLong(o, "maxInHash", -1);
+    f.maxVersionsUsedInHash = getLong(o, "maxVersionsUsedInHash", -1);
     f.versionsHash = getLong(o, "versionsHash", -1);
     f.numVersions = getLong(o, "numVersions", -1);
     f.numDocs = getLong(o, "numDocs", -1);


[30/40] lucene-solr:apiv2: SOLR-9269: Snapshots (Solr core level)

Posted by sa...@apache.org.
SOLR-9269: Snapshots (Solr core level)


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

Branch: refs/heads/apiv2
Commit: b7aa0b56bebcc22e63f0602a4ef15a728a799fc6
Parents: 7bf019a
Author: David Smiley <ds...@apache.org>
Authored: Thu Jul 28 14:23:09 2016 -0400
Committer: David Smiley <ds...@apache.org>
Committed: Thu Jul 28 14:23:09 2016 -0400

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   5 +
 .../solr/core/IndexDeletionPolicyWrapper.java   |  27 +-
 .../src/java/org/apache/solr/core/SolrCore.java |  36 +-
 .../core/snapshots/SolrSnapshotManager.java     | 134 ++++++
 .../snapshots/SolrSnapshotMetaDataManager.java  | 416 ++++++++++++++++++
 .../solr/core/snapshots/package-info.java       |  22 +
 .../org/apache/solr/handler/IndexFetcher.java   |  16 +-
 .../apache/solr/handler/ReplicationHandler.java |  24 +-
 .../org/apache/solr/handler/RestoreCore.java    |  18 +-
 .../org/apache/solr/handler/SnapShooter.java    |  39 +-
 .../solr/handler/admin/CoreAdminOperation.java  | 125 +++++-
 .../core/snapshots/TestSolrCoreSnapshots.java   | 419 +++++++++++++++++++
 .../apache/solr/handler/BackupRestoreUtils.java |  37 ++
 .../solr/handler/TestHdfsBackupRestoreCore.java |  46 +-
 .../solr/handler/TestReplicationHandler.java    |  20 +-
 .../client/solrj/request/CoreAdminRequest.java  |  57 +++
 .../solr/common/params/CoreAdminParams.java     |  10 +-
 17 files changed, 1365 insertions(+), 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b7aa0b56/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 6707e1a..3011cec 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -103,6 +103,11 @@ New Features
 * SOLR-9200: Add Delegation Token Support to Solr.
   (Gregory Chanan)
 
+* SOLR-9038: Solr core snapshots: The current commit can be snapshotted which retains the commit and associates it with
+  a name.  The core admin API can create snapshots, list them, and delete them. Snapshot names can be referenced in
+  doing a core backup, and in replication.  Snapshot metadata is stored in a new snapshot_metadata/ dir.
+  (Hrishikesh Gadre via David Smiley)
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b7aa0b56/solr/core/src/java/org/apache/solr/core/IndexDeletionPolicyWrapper.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/IndexDeletionPolicyWrapper.java b/solr/core/src/java/org/apache/solr/core/IndexDeletionPolicyWrapper.java
index 207c0e5..3616d4e 100644
--- a/solr/core/src/java/org/apache/solr/core/IndexDeletionPolicyWrapper.java
+++ b/solr/core/src/java/org/apache/solr/core/IndexDeletionPolicyWrapper.java
@@ -15,21 +15,26 @@
  * limitations under the License.
  */
 package org.apache.solr.core;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
 import org.apache.lucene.index.IndexCommit;
 import org.apache.lucene.index.IndexDeletionPolicy;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.store.Directory;
+import org.apache.solr.core.snapshots.SolrSnapshotMetaDataManager;
 import org.apache.solr.update.SolrIndexWriter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.util.*;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
 /**
  * A wrapper for an IndexDeletionPolicy instance.
  * <p>
@@ -52,9 +57,11 @@ public final class IndexDeletionPolicyWrapper extends IndexDeletionPolicy {
   private final Map<Long, Long> reserves = new ConcurrentHashMap<>();
   private volatile IndexCommit latestCommit;
   private final ConcurrentHashMap<Long, AtomicInteger> savedCommits = new ConcurrentHashMap<>();
+  private final SolrSnapshotMetaDataManager snapshotMgr;
 
-  public IndexDeletionPolicyWrapper(IndexDeletionPolicy deletionPolicy) {
+  public IndexDeletionPolicyWrapper(IndexDeletionPolicy deletionPolicy, SolrSnapshotMetaDataManager snapshotMgr) {
     this.deletionPolicy = deletionPolicy;
+    this.snapshotMgr = snapshotMgr;
   }
 
   /**
@@ -134,7 +141,6 @@ public final class IndexDeletionPolicyWrapper extends IndexDeletionPolicy {
     }
   }
 
-
   /**
    * Internal use for Lucene... do not explicitly call.
    */
@@ -185,7 +191,8 @@ public final class IndexDeletionPolicyWrapper extends IndexDeletionPolicy {
       Long gen = delegate.getGeneration();
       Long reserve = reserves.get(gen);
       if (reserve != null && System.nanoTime() < reserve) return;
-      if(savedCommits.containsKey(gen)) return;
+      if (savedCommits.containsKey(gen)) return;
+      if (snapshotMgr.isSnapshotted(gen)) return;
       delegate.delete();
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b7aa0b56/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 faef1c8..2704e4a 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -81,6 +81,7 @@ import org.apache.solr.common.util.ObjectReleaseTracker;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.DirectoryFactory.DirContext;
+import org.apache.solr.core.snapshots.SolrSnapshotMetaDataManager;
 import org.apache.solr.handler.IndexFetcher;
 import org.apache.solr.handler.ReplicationHandler;
 import org.apache.solr.handler.RequestHandlerBase;
@@ -184,6 +185,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
   private final Map<String,UpdateRequestProcessorChain> updateProcessorChains;
   private final Map<String, SolrInfoMBean> infoRegistry;
   private final IndexDeletionPolicyWrapper solrDelPolicy;
+  private final SolrSnapshotMetaDataManager snapshotMgr;
   private final DirectoryFactory directoryFactory;
   private IndexReaderFactory indexReaderFactory;
   private final Codec codec;
@@ -414,7 +416,19 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
     } else {
       delPolicy = new SolrDeletionPolicy();
     }
-    return new IndexDeletionPolicyWrapper(delPolicy);
+
+    return new IndexDeletionPolicyWrapper(delPolicy, snapshotMgr);
+  }
+
+  private SolrSnapshotMetaDataManager initSnapshotMetaDataManager() {
+    try {
+      String dirName = getDataDir() + SolrSnapshotMetaDataManager.SNAPSHOT_METADATA_DIR + "/";
+      Directory snapshotDir = directoryFactory.get(dirName, DirContext.DEFAULT,
+           getSolrConfig().indexConfig.lockType);
+      return new SolrSnapshotMetaDataManager(this, snapshotDir);
+    } catch (IOException e) {
+      throw new IllegalStateException(e);
+    }
   }
 
   private void initListeners() {
@@ -739,6 +753,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
 
       initListeners();
 
+      this.snapshotMgr = initSnapshotMetaDataManager();
       this.solrDelPolicy = initDeletionPolicy(delPolicy);
 
       this.codec = initCodec(solrConfig, this.schema);
@@ -1242,6 +1257,17 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
       }
     }
 
+    // Close the snapshots meta-data directory.
+    Directory snapshotsDir = snapshotMgr.getSnapshotsDir();
+    try {
+      this.directoryFactory.release(snapshotsDir);
+    }  catch (Throwable e) {
+      SolrException.log(log,e);
+      if (e instanceof Error) {
+        throw (Error) e;
+      }
+    }
+
     if (coreStateClosed) {
 
       try {
@@ -2343,6 +2369,14 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
     return solrDelPolicy;
   }
 
+  /**
+   * @return A reference of {@linkplain SolrSnapshotMetaDataManager}
+   *         managing the persistent snapshots for this Solr core.
+   */
+  public SolrSnapshotMetaDataManager getSnapshotMetaDataManager() {
+    return snapshotMgr;
+  }
+
   public ReentrantLock getRuleExpiryLock() {
     return ruleExpiryLock;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b7aa0b56/solr/core/src/java/org/apache/solr/core/snapshots/SolrSnapshotManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/snapshots/SolrSnapshotManager.java b/solr/core/src/java/org/apache/solr/core/snapshots/SolrSnapshotManager.java
new file mode 100644
index 0000000..95df3ff
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/core/snapshots/SolrSnapshotManager.java
@@ -0,0 +1,134 @@
+/*
+ * 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.snapshots;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexCommit;
+import org.apache.lucene.store.Directory;
+import org.apache.solr.core.snapshots.SolrSnapshotMetaDataManager.SnapshotMetaData;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class provides functionality required to handle the data files corresponding to Solr snapshots.
+ */
+public class SolrSnapshotManager {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  /**
+   * This method deletes index files of the {@linkplain IndexCommit} for the specified generation number.
+   *
+   * @param dir The index directory storing the snapshot.
+   * @param gen The generation number for the {@linkplain IndexCommit}
+   * @throws IOException in case of I/O errors.
+   */
+  public static void deleteIndexFiles ( Directory dir, Collection<SnapshotMetaData> snapshots, long gen ) throws IOException {
+    List<IndexCommit> commits = DirectoryReader.listCommits(dir);
+    Map<String, Integer> refCounts = buildRefCounts(snapshots, commits);
+    for (IndexCommit ic : commits) {
+      if (ic.getGeneration() == gen) {
+        deleteIndexFiles(dir,refCounts, ic);
+        break;
+      }
+    }
+  }
+
+  /**
+   * This method deletes all files not corresponding to a configured snapshot in the specified index directory.
+   *
+   * @param dir The index directory to search for.
+   * @throws IOException in case of I/O errors.
+   */
+  public static void deleteNonSnapshotIndexFiles (Directory dir, Collection<SnapshotMetaData> snapshots) throws IOException {
+    List<IndexCommit> commits = DirectoryReader.listCommits(dir);
+    Map<String, Integer> refCounts = buildRefCounts(snapshots, commits);
+    Set<Long> snapshotGenNumbers = snapshots.stream()
+                                            .map(SnapshotMetaData::getGenerationNumber)
+                                            .collect(Collectors.toSet());
+    for (IndexCommit ic : commits) {
+      if (!snapshotGenNumbers.contains(ic.getGeneration())) {
+        deleteIndexFiles(dir,refCounts, ic);
+      }
+    }
+  }
+
+  /**
+   * This method computes reference count for the index files by taking into consideration
+   * (a) configured snapshots and (b) files sharing between two or more {@linkplain IndexCommit} instances.
+   *
+   * @param snapshots A collection of user configured snapshots
+   * @param commits A list of {@linkplain IndexCommit} instances
+   * @return A map containing reference count for each index file referred in one of the {@linkplain IndexCommit} instances.
+   * @throws IOException in case of I/O error.
+   */
+  @VisibleForTesting
+  static Map<String, Integer> buildRefCounts (Collection<SnapshotMetaData> snapshots, List<IndexCommit> commits) throws IOException {
+    Map<String, Integer> result = new HashMap<>();
+    Map<Long, IndexCommit> commitsByGen = commits.stream().collect(
+        Collectors.toMap(IndexCommit::getGeneration, Function.identity()));
+
+    for(SnapshotMetaData md : snapshots) {
+      IndexCommit ic = commitsByGen.get(md.getGenerationNumber());
+      if (ic != null) {
+        Collection<String> fileNames = ic.getFileNames();
+        for(String fileName : fileNames) {
+          int refCount = result.getOrDefault(fileName, 0);
+          result.put(fileName, refCount+1);
+        }
+      }
+    }
+
+    return result;
+  }
+
+  /**
+   * This method deletes the index files associated with specified <code>indexCommit</code> provided they
+   * are not referred by some other {@linkplain IndexCommit}.
+   *
+   * @param dir The index directory containing the {@linkplain IndexCommit} to be deleted.
+   * @param refCounts A map containing reference counts for each file associated with every {@linkplain IndexCommit}
+   *                  in the specified directory.
+   * @param indexCommit The {@linkplain IndexCommit} whose files need to be deleted.
+   * @throws IOException in case of I/O errors.
+   */
+  private static void deleteIndexFiles ( Directory dir, Map<String, Integer> refCounts, IndexCommit indexCommit ) throws IOException {
+    log.info("Deleting index files for index commit with generation {} in directory {}", indexCommit.getGeneration(), dir);
+    for (String fileName : indexCommit.getFileNames()) {
+      try {
+        // Ensure that a file being deleted is not referred by some other commit.
+        int ref = refCounts.getOrDefault(fileName, 0);
+        log.debug("Reference count for file {} is {}", fileName, ref);
+        if (ref == 0) {
+          dir.deleteFile(fileName);
+        }
+      } catch (IOException e) {
+        log.warn("Unable to delete file {} in directory {} due to exception {}", fileName, dir, e.getMessage());
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b7aa0b56/solr/core/src/java/org/apache/solr/core/snapshots/SolrSnapshotMetaDataManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/snapshots/SolrSnapshotMetaDataManager.java b/solr/core/src/java/org/apache/solr/core/snapshots/SolrSnapshotMetaDataManager.java
new file mode 100644
index 0000000..26cbe21
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/core/snapshots/SolrSnapshotMetaDataManager.java
@@ -0,0 +1,416 @@
+/*
+ * 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.snapshots;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+import com.google.common.base.Preconditions;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexCommit;
+import org.apache.lucene.index.IndexDeletionPolicy;
+import org.apache.lucene.index.IndexWriterConfig.OpenMode;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.IOUtils;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.core.DirectoryFactory;
+import org.apache.solr.core.DirectoryFactory.DirContext;
+import org.apache.solr.core.IndexDeletionPolicyWrapper;
+import org.apache.solr.core.SolrCore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class is responsible to manage the persistent snapshots meta-data for the Solr indexes. The
+ * persistent snapshots are implemented by relying on Lucene {@linkplain IndexDeletionPolicy}
+ * abstraction to configure a specific {@linkplain IndexCommit} to be retained. The
+ * {@linkplain IndexDeletionPolicyWrapper} in Solr uses this class to create/delete the Solr index
+ * snapshots.
+ */
+public class SolrSnapshotMetaDataManager {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  public static final String SNAPSHOT_METADATA_DIR = "snapshot_metadata";
+
+  /**
+   * A class defining the meta-data for a specific snapshot.
+   */
+  public static class SnapshotMetaData {
+    private String name;
+    private String indexDirPath;
+    private long generationNumber;
+
+    public SnapshotMetaData(String name, String indexDirPath, long generationNumber) {
+      super();
+      this.name = name;
+      this.indexDirPath = indexDirPath;
+      this.generationNumber = generationNumber;
+    }
+
+    public String getName() {
+      return name;
+    }
+
+    public String getIndexDirPath() {
+      return indexDirPath;
+    }
+
+    public long getGenerationNumber() {
+      return generationNumber;
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder builder = new StringBuilder();
+      builder.append("SnapshotMetaData[name=");
+      builder.append(name);
+      builder.append(", indexDirPath=");
+      builder.append(indexDirPath);
+      builder.append(", generation=");
+      builder.append(generationNumber);
+      builder.append("]");
+      return builder.toString();
+    }
+  }
+
+  /** Prefix used for the save file. */
+  public static final String SNAPSHOTS_PREFIX = "snapshots_";
+  private static final int VERSION_START = 0;
+  private static final int VERSION_CURRENT = VERSION_START;
+  private static final String CODEC_NAME = "solr-snapshots";
+
+  // The index writer which maintains the snapshots metadata
+  private long nextWriteGen;
+
+  private final Directory dir;
+
+  /** Used to map snapshot name to snapshot meta-data. */
+  protected final Map<String,SnapshotMetaData> nameToDetailsMapping = new LinkedHashMap<>();
+  /** Used to figure out the *current* index data directory path */
+  private final SolrCore solrCore;
+
+  /**
+   * A constructor.
+   *
+   * @param dir The directory where the snapshot meta-data should be stored. Enables updating
+   *            the existing meta-data.
+   * @throws IOException in case of errors.
+   */
+  public SolrSnapshotMetaDataManager(SolrCore solrCore, Directory dir) throws IOException {
+    this(solrCore, dir, OpenMode.CREATE_OR_APPEND);
+  }
+
+  /**
+   * A constructor.
+   *
+   * @param dir The directory where the snapshot meta-data is stored.
+   * @param mode CREATE If previous meta-data should be erased.
+   *             APPEND If previous meta-data should be read and updated.
+   *             CREATE_OR_APPEND Creates a new meta-data structure if one does not exist
+   *                              Updates the existing structure if one exists.
+   * @throws IOException in case of errors.
+   */
+  public SolrSnapshotMetaDataManager(SolrCore solrCore, Directory dir, OpenMode mode) throws IOException {
+    this.solrCore = solrCore;
+    this.dir = dir;
+
+    if (mode == OpenMode.CREATE) {
+      deleteSnapshotMetadataFiles();
+    }
+
+    loadFromSnapshotMetadataFile();
+
+    if (mode == OpenMode.APPEND && nextWriteGen == 0) {
+      throw new IllegalStateException("no snapshots stored in this directory");
+    }
+  }
+
+  /**
+   * @return The snapshot meta-data directory
+   */
+  public Directory getSnapshotsDir() {
+    return dir;
+  }
+
+  /**
+   * This method creates a new snapshot meta-data entry.
+   *
+   * @param name The name of the snapshot.
+   * @param indexDirPath The directory path where the index files are stored.
+   * @param gen The generation number for the {@linkplain IndexCommit} being snapshotted.
+   * @throws IOException in case of I/O errors.
+   */
+  public synchronized void snapshot(String name, String indexDirPath, long gen) throws IOException {
+    Preconditions.checkNotNull(name);
+
+    log.info("Creating the snapshot named {} for core {} associated with index commit with generation {} in directory {}"
+        , name, solrCore.getName(), gen, indexDirPath);
+
+    if(nameToDetailsMapping.containsKey(name)) {
+      throw new SolrException(ErrorCode.BAD_REQUEST, "A snapshot with name " + name + " already exists");
+    }
+
+    SnapshotMetaData d = new SnapshotMetaData(name, indexDirPath, gen);
+    nameToDetailsMapping.put(name, d);
+
+    boolean success = false;
+    try {
+      persist();
+      success = true;
+    } finally {
+      if (!success) {
+        try {
+          release(name);
+        } catch (Exception e) {
+          // Suppress so we keep throwing original exception
+        }
+      }
+    }
+  }
+
+  /**
+   * This method deletes a previously created snapshot (if any).
+   *
+   * @param name The name of the snapshot to be deleted.
+   * @return The snapshot meta-data if the snapshot with the snapshot name exists.
+   * @throws IOException in case of I/O error
+   */
+  public synchronized Optional<SnapshotMetaData> release(String name) throws IOException {
+    log.info("Deleting the snapshot named {} for core {}", name, solrCore.getName());
+    SnapshotMetaData result = nameToDetailsMapping.remove(Preconditions.checkNotNull(name));
+    if(result != null) {
+      boolean success = false;
+      try {
+        persist();
+        success = true;
+      } finally {
+        if (!success) {
+          nameToDetailsMapping.put(name, result);
+        }
+      }
+    }
+    return Optional.ofNullable(result);
+  }
+
+  /**
+   * This method returns if snapshot is created for the specified generation number in
+   * the *current* index directory.
+   *
+   * @param genNumber The generation number for the {@linkplain IndexCommit} to be checked.
+   * @return true if the snapshot is created.
+   *         false otherwise.
+   */
+  public synchronized boolean isSnapshotted(long genNumber) {
+    return !nameToDetailsMapping.isEmpty() && isSnapshotted(solrCore.getIndexDir(), genNumber);
+  }
+
+  /**
+   * This method returns if snapshot is created for the specified generation number in
+   * the specified index directory.
+   *
+   * @param genNumber The generation number for the {@linkplain IndexCommit} to be checked.
+   * @return true if the snapshot is created.
+   *         false otherwise.
+   */
+  public synchronized boolean isSnapshotted(String indexDirPath, long genNumber) {
+    return !nameToDetailsMapping.isEmpty()
+        && nameToDetailsMapping.values().stream()
+           .anyMatch(entry -> entry.getIndexDirPath().equals(indexDirPath) && entry.getGenerationNumber() == genNumber);
+  }
+
+  /**
+   * This method returns the snapshot meta-data for the specified name (if it exists).
+   *
+   * @param name The name of the snapshot
+   * @return The snapshot meta-data if exists.
+   */
+  public synchronized Optional<SnapshotMetaData> getSnapshotMetaData(String name) {
+    return Optional.ofNullable(nameToDetailsMapping.get(name));
+  }
+
+  /**
+   * @return A list of snapshots created so far.
+   */
+  public synchronized List<String> listSnapshots() {
+    // We create a copy for thread safety.
+    return new ArrayList<>(nameToDetailsMapping.keySet());
+  }
+
+  /**
+   * This method returns a list of snapshots created in a specified index directory.
+   *
+   * @param indexDirPath The index directory path.
+   * @return a list snapshots stored in the specified directory.
+   */
+  public synchronized Collection<SnapshotMetaData> listSnapshotsInIndexDir(String indexDirPath) {
+    return nameToDetailsMapping.values().stream()
+        .filter(entry -> indexDirPath.equals(entry.getIndexDirPath()))
+        .collect(Collectors.toList());
+  }
+
+  /**
+   * This method returns the {@linkplain IndexCommit} associated with the specified
+   * <code>commitName</code>. A snapshot with specified <code>commitName</code> must
+   * be created before invoking this method.
+   *
+   * @param commitName The name of persisted commit
+   * @return the {@linkplain IndexCommit}
+   * @throws IOException in case of I/O error.
+   */
+  public Optional<IndexCommit> getIndexCommitByName(String commitName) throws IOException {
+    Optional<IndexCommit> result = Optional.empty();
+    Optional<SnapshotMetaData> metaData = getSnapshotMetaData(commitName);
+    if (metaData.isPresent()) {
+      String indexDirPath = metaData.get().getIndexDirPath();
+      long gen = metaData.get().getGenerationNumber();
+
+      Directory d = solrCore.getDirectoryFactory().get(indexDirPath, DirContext.DEFAULT, DirectoryFactory.LOCK_TYPE_NONE);
+      try {
+        result = DirectoryReader.listCommits(d)
+                                .stream()
+                                .filter(ic -> ic.getGeneration() == gen)
+                                .findAny();
+
+        if (!result.isPresent()) {
+          log.warn("Unable to find commit with generation {} in the directory {}", gen, indexDirPath);
+        }
+
+      } finally {
+        solrCore.getDirectoryFactory().release(d);
+      }
+    } else {
+      log.warn("Commit with name {} is not persisted for core {}", commitName, solrCore.getName());
+    }
+
+    return result;
+  }
+
+  private synchronized void persist() throws IOException {
+    String fileName = SNAPSHOTS_PREFIX + nextWriteGen;
+    IndexOutput out = dir.createOutput(fileName, IOContext.DEFAULT);
+    boolean success = false;
+    try {
+      CodecUtil.writeHeader(out, CODEC_NAME, VERSION_CURRENT);
+      out.writeVInt(nameToDetailsMapping.size());
+      for(Entry<String,SnapshotMetaData> ent : nameToDetailsMapping.entrySet()) {
+        out.writeString(ent.getKey());
+        out.writeString(ent.getValue().getIndexDirPath());
+        out.writeVLong(ent.getValue().getGenerationNumber());
+      }
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(out);
+        IOUtils.deleteFilesIgnoringExceptions(dir, fileName);
+      } else {
+        IOUtils.close(out);
+      }
+    }
+
+    dir.sync(Collections.singletonList(fileName));
+
+    if (nextWriteGen > 0) {
+      String lastSaveFile = SNAPSHOTS_PREFIX + (nextWriteGen-1);
+      // exception OK: likely it didn't exist
+      IOUtils.deleteFilesIgnoringExceptions(dir, lastSaveFile);
+    }
+
+    nextWriteGen++;
+  }
+
+  private synchronized void deleteSnapshotMetadataFiles() throws IOException {
+    for(String file : dir.listAll()) {
+      if (file.startsWith(SNAPSHOTS_PREFIX)) {
+        dir.deleteFile(file);
+      }
+    }
+  }
+
+  /**
+   * Reads the snapshot meta-data information from the given {@link Directory}.
+   */
+  private synchronized void loadFromSnapshotMetadataFile() throws IOException {
+    log.info("Loading from snapshot metadata file...");
+    long genLoaded = -1;
+    IOException ioe = null;
+    List<String> snapshotFiles = new ArrayList<>();
+    for(String file : dir.listAll()) {
+      if (file.startsWith(SNAPSHOTS_PREFIX)) {
+        long gen = Long.parseLong(file.substring(SNAPSHOTS_PREFIX.length()));
+        if (genLoaded == -1 || gen > genLoaded) {
+          snapshotFiles.add(file);
+          Map<String, SnapshotMetaData> snapshotMetaDataMapping = new HashMap<>();
+          IndexInput in = dir.openInput(file, IOContext.DEFAULT);
+          try {
+            CodecUtil.checkHeader(in, CODEC_NAME, VERSION_START, VERSION_START);
+            int count = in.readVInt();
+            for(int i=0;i<count;i++) {
+              String name = in.readString();
+              String indexDirPath = in.readString();
+              long commitGen = in.readVLong();
+              snapshotMetaDataMapping.put(name, new SnapshotMetaData(name, indexDirPath, commitGen));
+            }
+          } catch (IOException ioe2) {
+            // Save first exception & throw in the end
+            if (ioe == null) {
+              ioe = ioe2;
+            }
+          } finally {
+            in.close();
+          }
+
+          genLoaded = gen;
+          nameToDetailsMapping.clear();
+          nameToDetailsMapping.putAll(snapshotMetaDataMapping);
+        }
+      }
+    }
+
+    if (genLoaded == -1) {
+      // Nothing was loaded...
+      if (ioe != null) {
+        // ... not for lack of trying:
+        throw ioe;
+      }
+    } else {
+      if (snapshotFiles.size() > 1) {
+        // Remove any broken / old snapshot files:
+        String curFileName = SNAPSHOTS_PREFIX + genLoaded;
+        for(String file : snapshotFiles) {
+          if (!curFileName.equals(file)) {
+            IOUtils.deleteFilesIgnoringExceptions(dir, file);
+          }
+        }
+      }
+      nextWriteGen = 1+genLoaded;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b7aa0b56/solr/core/src/java/org/apache/solr/core/snapshots/package-info.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/snapshots/package-info.java b/solr/core/src/java/org/apache/solr/core/snapshots/package-info.java
new file mode 100644
index 0000000..3242cd3
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/core/snapshots/package-info.java
@@ -0,0 +1,22 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+
+/**
+ * Core classes for Solr's persistent snapshots functionality
+ */
+package org.apache.solr.core.snapshots;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b7aa0b56/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 714b800..77624c9 100644
--- a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
+++ b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
@@ -82,6 +82,9 @@ import org.apache.solr.core.DirectoryFactory;
 import org.apache.solr.core.DirectoryFactory.DirContext;
 import org.apache.solr.core.IndexDeletionPolicyWrapper;
 import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.snapshots.SolrSnapshotManager;
+import org.apache.solr.core.snapshots.SolrSnapshotMetaDataManager;
+import org.apache.solr.core.snapshots.SolrSnapshotMetaDataManager.SnapshotMetaData;
 import org.apache.solr.handler.ReplicationHandler.*;
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
@@ -468,9 +471,18 @@ public class IndexFetcher {
                 // let the system know we are changing dir's and the old one
                 // may be closed
                 if (indexDir != null) {
-                  LOG.info("removing old index directory " + indexDir);
                   solrCore.getDirectoryFactory().doneWithDirectory(indexDir);
-                  solrCore.getDirectoryFactory().remove(indexDir);
+
+                  SolrSnapshotMetaDataManager snapshotsMgr = solrCore.getSnapshotMetaDataManager();
+                  Collection<SnapshotMetaData> snapshots = snapshotsMgr.listSnapshotsInIndexDir(indexDirPath);
+
+                  // Delete the old index directory only if no snapshot exists in that directory.
+                  if(snapshots.isEmpty()) {
+                    LOG.info("removing old index directory " + indexDir);
+                    solrCore.getDirectoryFactory().remove(indexDir);
+                  } else {
+                    SolrSnapshotManager.deleteNonSnapshotIndexFiles(indexDir, snapshots);
+                  }
                 }
               }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b7aa0b56/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
index ff93c42..0870e35 100644
--- a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
@@ -87,6 +87,7 @@ import org.apache.solr.core.SolrDeletionPolicy;
 import org.apache.solr.core.SolrEventListener;
 import org.apache.solr.core.backup.repository.BackupRepository;
 import org.apache.solr.core.backup.repository.LocalFileSystemRepository;
+import org.apache.solr.core.snapshots.SolrSnapshotMetaDataManager;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.search.SolrIndexSearcher;
@@ -512,11 +513,24 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
         numberToKeep = Integer.MAX_VALUE;
       }
 
-      IndexDeletionPolicyWrapper delPolicy = core.getDeletionPolicy();
-      IndexCommit indexCommit = delPolicy.getLatestCommit();
+      IndexCommit indexCommit = null;
+      String commitName = params.get(CoreAdminParams.COMMIT_NAME);
+      if (commitName != null) {
+        SolrSnapshotMetaDataManager snapshotMgr = core.getSnapshotMetaDataManager();
+        Optional<IndexCommit> commit = snapshotMgr.getIndexCommitByName(commitName);
+        if(commit.isPresent()) {
+          indexCommit = commit.get();
+        } else {
+          throw new SolrException(ErrorCode.BAD_REQUEST, "Unable to find an index commit with name " + commitName +
+              " for core " + core.getName());
+        }
+      } else {
+        IndexDeletionPolicyWrapper delPolicy = core.getDeletionPolicy();
+        indexCommit = delPolicy.getLatestCommit();
 
-      if (indexCommit == null) {
-        indexCommit = req.getSearcher().getIndexReader().getIndexCommit();
+        if (indexCommit == null) {
+          indexCommit = req.getSearcher().getIndexReader().getIndexCommit();
+        }
       }
 
       String location = params.get(CoreAdminParams.BACKUP_LOCATION);
@@ -539,7 +553,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
       }
 
       // small race here before the commit point is saved
-      SnapShooter snapShooter = new SnapShooter(repo, core, location, params.get(NAME));
+      SnapShooter snapShooter = new SnapShooter(repo, core, location, params.get(NAME), commitName);
       snapShooter.validateCreateSnapshot();
       snapShooter.createSnapAsync(indexCommit, numberToKeep, (nl) -> snapShootDetails = nl);
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b7aa0b56/solr/core/src/java/org/apache/solr/handler/RestoreCore.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/RestoreCore.java b/solr/core/src/java/org/apache/solr/handler/RestoreCore.java
index d3c98fa..6aef35c 100644
--- a/solr/core/src/java/org/apache/solr/handler/RestoreCore.java
+++ b/solr/core/src/java/org/apache/solr/handler/RestoreCore.java
@@ -19,6 +19,7 @@ package org.apache.solr.handler;
 import java.lang.invoke.MethodHandles;
 import java.net.URI;
 import java.text.SimpleDateFormat;
+import java.util.Collection;
 import java.util.Date;
 import java.util.Locale;
 import java.util.concurrent.Callable;
@@ -32,6 +33,9 @@ import org.apache.solr.common.SolrException;
 import org.apache.solr.core.DirectoryFactory;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.backup.repository.BackupRepository;
+import org.apache.solr.core.snapshots.SolrSnapshotManager;
+import org.apache.solr.core.snapshots.SolrSnapshotMetaDataManager;
+import org.apache.solr.core.snapshots.SolrSnapshotMetaDataManager.SnapshotMetaData;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -63,6 +67,7 @@ public class RestoreCore implements Callable<Boolean> {
     String restoreIndexName = "restore." + dateFormat.format(new Date());
     String restoreIndexPath = core.getDataDir() + restoreIndexName;
 
+    String indexDirPath = core.getIndexDir();
     Directory restoreIndexDir = null;
     Directory indexDir = null;
     try {
@@ -71,7 +76,7 @@ public class RestoreCore implements Callable<Boolean> {
           DirectoryFactory.DirContext.DEFAULT, core.getSolrConfig().indexConfig.lockType);
 
       //Prefer local copy.
-      indexDir = core.getDirectoryFactory().get(core.getIndexDir(),
+      indexDir = core.getDirectoryFactory().get(indexDirPath,
           DirectoryFactory.DirContext.DEFAULT, core.getSolrConfig().indexConfig.lockType);
 
       //Move all files from backupDir to restoreIndexDir
@@ -130,7 +135,16 @@ public class RestoreCore implements Callable<Boolean> {
       }
       if (success) {
         core.getDirectoryFactory().doneWithDirectory(indexDir);
-        core.getDirectoryFactory().remove(indexDir);
+
+        SolrSnapshotMetaDataManager snapshotsMgr = core.getSnapshotMetaDataManager();
+        Collection<SnapshotMetaData> snapshots = snapshotsMgr.listSnapshotsInIndexDir(indexDirPath);
+
+        // Delete the old index directory only if no snapshot exists in that directory.
+        if (snapshots.isEmpty()) {
+          core.getDirectoryFactory().remove(indexDir);
+        } else {
+          SolrSnapshotManager.deleteNonSnapshotIndexFiles(indexDir, snapshots);
+        }
       }
 
       return true;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b7aa0b56/solr/core/src/java/org/apache/solr/handler/SnapShooter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/SnapShooter.java b/solr/core/src/java/org/apache/solr/handler/SnapShooter.java
index 5ac3243..e12649d 100644
--- a/solr/core/src/java/org/apache/solr/handler/SnapShooter.java
+++ b/solr/core/src/java/org/apache/solr/handler/SnapShooter.java
@@ -26,12 +26,14 @@ import java.util.Collections;
 import java.util.Date;
 import java.util.List;
 import java.util.Locale;
+import java.util.Optional;
 import java.util.function.Consumer;
 
 import com.google.common.base.Preconditions;
 import org.apache.lucene.index.IndexCommit;
 import org.apache.lucene.store.Directory;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.DirectoryFactory.DirContext;
 import org.apache.solr.core.IndexDeletionPolicyWrapper;
@@ -39,6 +41,7 @@ import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.backup.repository.BackupRepository;
 import org.apache.solr.core.backup.repository.BackupRepository.PathType;
 import org.apache.solr.core.backup.repository.LocalFileSystemRepository;
+import org.apache.solr.core.snapshots.SolrSnapshotMetaDataManager;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.util.RefCounted;
 import org.slf4j.Logger;
@@ -59,6 +62,7 @@ public class SnapShooter {
   private URI baseSnapDirPath = null;
   private URI snapshotDirPath = null;
   private BackupRepository backupRepo = null;
+  private String commitName; // can be null
 
   @Deprecated
   public SnapShooter(SolrCore core, String location, String snapshotName) {
@@ -71,14 +75,14 @@ public class SnapShooter {
     } else {
       snapDirStr = core.getCoreDescriptor().getInstanceDir().resolve(location).normalize().toString();
     }
-    initialize(new LocalFileSystemRepository(), core, snapDirStr, snapshotName);
+    initialize(new LocalFileSystemRepository(), core, snapDirStr, snapshotName, null);
   }
 
-  public SnapShooter(BackupRepository backupRepo, SolrCore core, String location, String snapshotName) {
-    initialize(backupRepo, core, location, snapshotName);
+  public SnapShooter(BackupRepository backupRepo, SolrCore core, String location, String snapshotName, String commitName) {
+    initialize(backupRepo, core, location, snapshotName, commitName);
   }
 
-  private void initialize(BackupRepository backupRepo, SolrCore core, String location, String snapshotName) {
+  private void initialize(BackupRepository backupRepo, SolrCore core, String location, String snapshotName, String commitName) {
     this.solrCore = Preconditions.checkNotNull(core);
     this.backupRepo = Preconditions.checkNotNull(backupRepo);
     this.baseSnapDirPath = backupRepo.createURI(Preconditions.checkNotNull(location)).normalize();
@@ -90,6 +94,7 @@ public class SnapShooter {
       directoryName = "snapshot." + fmt.format(new Date());
     }
     this.snapshotDirPath = backupRepo.createURI(location, directoryName);
+    this.commitName = commitName;
   }
 
   public BackupRepository getBackupRepository() {
@@ -145,16 +150,26 @@ public class SnapShooter {
   }
 
   public NamedList createSnapshot() throws Exception {
-    IndexDeletionPolicyWrapper deletionPolicy = solrCore.getDeletionPolicy();
     RefCounted<SolrIndexSearcher> searcher = solrCore.getSearcher();
     try {
-      //TODO should we try solrCore.getDeletionPolicy().getLatestCommit() first?
-      IndexCommit indexCommit = searcher.get().getIndexReader().getIndexCommit();
-      deletionPolicy.saveCommitPoint(indexCommit.getGeneration());
-      try {
-        return createSnapshot(indexCommit);
-      } finally {
-        deletionPolicy.releaseCommitPoint(indexCommit.getGeneration());
+      if (commitName != null) {
+        SolrSnapshotMetaDataManager snapshotMgr = solrCore.getSnapshotMetaDataManager();
+        Optional<IndexCommit> commit = snapshotMgr.getIndexCommitByName(commitName);
+        if(commit.isPresent()) {
+          return createSnapshot(commit.get());
+        }
+        throw new SolrException(ErrorCode.SERVER_ERROR, "Unable to find an index commit with name " + commitName +
+            " for core " + solrCore.getName());
+      } else {
+        //TODO should we try solrCore.getDeletionPolicy().getLatestCommit() first?
+        IndexDeletionPolicyWrapper deletionPolicy = solrCore.getDeletionPolicy();
+        IndexCommit indexCommit = searcher.get().getIndexReader().getIndexCommit();
+        deletionPolicy.saveCommitPoint(indexCommit.getGeneration());
+        try {
+          return createSnapshot(indexCommit);
+        } finally {
+          deletionPolicy.releaseCommitPoint(indexCommit.getGeneration());
+        }
       }
     } finally {
       searcher.decref();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b7aa0b56/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java
index 9b9aafa..e4103c5 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java
@@ -34,6 +34,7 @@ import java.util.concurrent.Future;
 import com.google.common.collect.Lists;
 import org.apache.commons.lang.StringUtils;
 import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexCommit;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.IOUtils;
@@ -59,9 +60,13 @@ import org.apache.solr.core.CachingDirectoryFactory;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.core.DirectoryFactory;
+import org.apache.solr.core.DirectoryFactory.DirContext;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.core.backup.repository.BackupRepository;
+import org.apache.solr.core.snapshots.SolrSnapshotManager;
+import org.apache.solr.core.snapshots.SolrSnapshotMetaDataManager;
+import org.apache.solr.core.snapshots.SolrSnapshotMetaDataManager.SnapshotMetaData;
 import org.apache.solr.handler.RestoreCore;
 import org.apache.solr.handler.SnapShooter;
 import org.apache.solr.handler.admin.CoreAdminHandler.CoreAdminOp;
@@ -794,22 +799,26 @@ enum CoreAdminOperation implements CoreAdminOp {
           + " parameter or as a default repository property");
     }
 
-    try (SolrCore core = it.handler.coreContainer.getCore(cname)) {
-      SnapShooter snapShooter = new SnapShooter(repository, core, location, name);
-      // validateCreateSnapshot will create parent dirs instead of throw; that choice is dubious.
-      //  But we want to throw. One reason is that
-      //  this dir really should, in fact must, already exist here if triggered via a collection backup on a shared
-      //  file system. Otherwise, perhaps the FS location isn't shared -- we want an error.
-      if (!snapShooter.getBackupRepository().exists(snapShooter.getLocation())) {
-        throw new SolrException(ErrorCode.BAD_REQUEST,
-            "Directory to contain snapshots doesn't exist: " + snapShooter.getLocation());
+      // An optional parameter to describe the snapshot to be backed-up. If this
+      // parameter is not supplied, the latest index commit is backed-up.
+      String commitName = params.get(CoreAdminParams.COMMIT_NAME);
+
+      try (SolrCore core = it.handler.coreContainer.getCore(cname)) {
+        SnapShooter snapShooter = new SnapShooter(repository, core, location, name, commitName);
+        // validateCreateSnapshot will create parent dirs instead of throw; that choice is dubious.
+        //  But we want to throw. One reason is that
+        //  this dir really should, in fact must, already exist here if triggered via a collection backup on a shared
+        //  file system. Otherwise, perhaps the FS location isn't shared -- we want an error.
+        if (!snapShooter.getBackupRepository().exists(snapShooter.getLocation())) {
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+              "Directory to contain snapshots doesn't exist: " + snapShooter.getLocation());
+        }
+        snapShooter.validateCreateSnapshot();
+        snapShooter.createSnapshot();
+      } catch (Exception e) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+            "Failed to backup core=" + cname + " because " + e, e);
       }
-      snapShooter.validateCreateSnapshot();
-      snapShooter.createSnapshot();
-    } catch (Exception e) {
-      throw new SolrException(ErrorCode.SERVER_ERROR,
-          "Failed to backup core=" + cname + " because " + e, e);
-    }
   }),
 
   RESTORECORE_OP(RESTORECORE, it -> {
@@ -845,6 +854,92 @@ enum CoreAdminOperation implements CoreAdminOp {
         throw new SolrException(ErrorCode.SERVER_ERROR, "Failed to restore core=" + core.getName());
       }
     }
+  }),
+  CREATESNAPSHOT_OP(CREATESNAPSHOT, it -> {
+    CoreContainer cc = it.handler.getCoreContainer();
+    final SolrParams params = it.req.getParams();
+
+    String commitName = params.required().get(CoreAdminParams.COMMIT_NAME);
+    String cname = params.required().get(CoreAdminParams.CORE);
+    try (SolrCore core = cc.getCore(cname)) {
+      if (core == null) {
+        throw new SolrException(ErrorCode.BAD_REQUEST, "Unable to locate core " + cname);
+      }
+
+      String indexDirPath = core.getIndexDir();
+      IndexCommit ic = core.getDeletionPolicy().getLatestCommit();
+      if (ic == null) {
+        RefCounted<SolrIndexSearcher> searcher = core.getSearcher();
+        try {
+          ic = searcher.get().getIndexReader().getIndexCommit();
+        } finally {
+          searcher.decref();
+        }
+      }
+      SolrSnapshotMetaDataManager mgr = core.getSnapshotMetaDataManager();
+      mgr.snapshot(commitName, indexDirPath, ic.getGeneration());
+
+      it.rsp.add("core", core.getName());
+      it.rsp.add("commitName", commitName);
+      it.rsp.add("indexDirPath", indexDirPath);
+      it.rsp.add("generation", ic.getGeneration());
+    }
+  }),
+  DELETESNAPSHOT_OP(DELETESNAPSHOT, it -> {
+    CoreContainer cc = it.handler.getCoreContainer();
+    final SolrParams params = it.req.getParams();
+
+    String commitName = params.required().get(CoreAdminParams.COMMIT_NAME);
+    String cname = params.required().get(CoreAdminParams.CORE);
+    try (SolrCore core = cc.getCore(cname)) {
+      if (core == null) {
+        throw new SolrException(ErrorCode.BAD_REQUEST, "Unable to locate core " + cname);
+      }
+
+      SolrSnapshotMetaDataManager mgr = core.getSnapshotMetaDataManager();
+      Optional<SnapshotMetaData> metadata = mgr.release(commitName);
+      if (metadata.isPresent()) {
+        long gen = metadata.get().getGenerationNumber();
+        String indexDirPath = metadata.get().getIndexDirPath();
+
+        // If the directory storing the snapshot is not the same as the *current* core
+        // index directory, then delete the files corresponding to this snapshot.
+        // Otherwise we leave the index files related to snapshot as is (assuming the
+        // underlying Solr IndexDeletionPolicy will clean them up appropriately).
+        if (!indexDirPath.equals(core.getIndexDir())) {
+          Directory d = core.getDirectoryFactory().get(indexDirPath, DirContext.DEFAULT, DirectoryFactory.LOCK_TYPE_NONE);
+          try {
+            SolrSnapshotManager.deleteIndexFiles(d, mgr.listSnapshotsInIndexDir(indexDirPath), gen);
+          } finally {
+            core.getDirectoryFactory().release(d);
+          }
+        }
+      }
+    }
+  }),
+  LISTSNAPSHOTS_OP(LISTSNAPSHOTS, it -> {
+    CoreContainer cc = it.handler.getCoreContainer();
+    final SolrParams params = it.req.getParams();
+
+    String cname = params.required().get(CoreAdminParams.CORE);
+    try ( SolrCore core = cc.getCore(cname) ) {
+      if (core == null) {
+        throw new SolrException(ErrorCode.BAD_REQUEST, "Unable to locate core " + cname);
+      }
+
+      SolrSnapshotMetaDataManager mgr = core.getSnapshotMetaDataManager();
+      NamedList result = new NamedList();
+      for (String name : mgr.listSnapshots()) {
+        Optional<SnapshotMetaData> metadata = mgr.getSnapshotMetaData(name);
+        if ( metadata.isPresent() ) {
+          NamedList<String> props = new NamedList<>();
+          props.add("generation", String.valueOf(metadata.get().getGenerationNumber()));
+          props.add("indexDirPath", metadata.get().getIndexDirPath());
+          result.add(name, props);
+        }
+      }
+      it.rsp.add("snapshots", result);
+    }
   });
 
   final CoreAdminParams.CoreAdminAction action;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b7aa0b56/solr/core/src/test/org/apache/solr/core/snapshots/TestSolrCoreSnapshots.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/snapshots/TestSolrCoreSnapshots.java b/solr/core/src/test/org/apache/solr/core/snapshots/TestSolrCoreSnapshots.java
new file mode 100644
index 0000000..aacac52
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/core/snapshots/TestSolrCoreSnapshots.java
@@ -0,0 +1,419 @@
+/*
+ * 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.snapshots;
+
+import java.lang.invoke.MethodHandles;
+import java.nio.file.Files;
+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.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexCommit;
+import org.apache.lucene.index.IndexNotFoundException;
+import org.apache.lucene.store.SimpleFSDirectory;
+import org.apache.lucene.util.LuceneTestCase.Slow;
+import org.apache.lucene.util.TestUtil;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.CreateSnapshot;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.DeleteSnapshot;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.ListSnapshots;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.SolrInputDocument;
+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.CoreAdminAction;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.snapshots.SolrSnapshotMetaDataManager.SnapshotMetaData;
+import org.apache.solr.handler.BackupRestoreUtils;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
+
+@SolrTestCaseJ4.SuppressSSL // Currently unknown why SSL does not work with this test
+@Slow
+public class TestSolrCoreSnapshots extends SolrCloudTestCase {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  private static long docsSeed; // see indexDocs()
+
+  @BeforeClass
+  public static void setupClass() throws Exception {
+    useFactory("solr.StandardDirectoryFactory");
+    configureCluster(1)// nodes
+        .addConfig("conf1", TEST_PATH().resolve("configsets").resolve("cloud-minimal").resolve("conf"))
+        .configure();
+
+    docsSeed = random().nextLong();
+  }
+
+  @AfterClass
+  public static void teardownClass() throws Exception {
+    System.clearProperty("test.build.data");
+    System.clearProperty("test.cache.data");
+  }
+
+  @Test
+  public void testBackupRestore() throws Exception {
+    CloudSolrClient solrClient = cluster.getSolrClient();
+    String collectionName = "SolrCoreSnapshots";
+    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName, "conf1", 1, 1);
+    create.process(solrClient);
+
+    String location = createTempDir().toFile().getAbsolutePath();
+    int nDocs = BackupRestoreUtils.indexDocs(cluster.getSolrClient(), collectionName, docsSeed);
+
+    DocCollection collectionState = solrClient.getZkStateReader().getClusterState().getCollection(collectionName);
+    assertEquals(1, collectionState.getActiveSlices().size());
+    Slice shard = collectionState.getActiveSlices().iterator().next();
+    assertEquals(1, shard.getReplicas().size());
+    Replica replica = shard.getReplicas().iterator().next();
+
+    String replicaBaseUrl = replica.getStr(BASE_URL_PROP);
+    String coreName = replica.getStr(ZkStateReader.CORE_NAME_PROP);
+    String backupName = TestUtil.randomSimpleString(random(), 1, 5);
+    String commitName = TestUtil.randomSimpleString(random(), 1, 5);
+    String duplicateName = commitName.concat("_duplicate");
+
+    try (
+        SolrClient adminClient = getHttpSolrClient(cluster.getJettySolrRunners().get(0).getBaseUrl().toString());
+        SolrClient masterClient = getHttpSolrClient(replica.getCoreUrl())) {
+
+      SnapshotMetaData metaData = createSnapshot(adminClient, coreName, commitName);
+      // Create another snapshot referring to the same index commit to verify the
+      // reference counting implementation during snapshot deletion.
+      SnapshotMetaData duplicateCommit = createSnapshot(adminClient, coreName, duplicateName);
+
+      assertEquals (metaData.getIndexDirPath(), duplicateCommit.getIndexDirPath());
+      assertEquals (metaData.getGenerationNumber(), duplicateCommit.getGenerationNumber());
+
+      // Delete all documents
+      masterClient.deleteByQuery("*:*");
+      masterClient.commit();
+      BackupRestoreUtils.verifyDocs(0, cluster.getSolrClient(), collectionName);
+
+      // Verify that the index directory contains at least 2 index commits - one referred by the snapshots
+      // and the other containing document deletions.
+      {
+        List<IndexCommit> commits = listCommits(metaData.getIndexDirPath());
+        assertTrue(2 <= commits.size());
+      }
+
+      // Backup the earlier created snapshot.
+      {
+        Map<String,String> params = new HashMap<>();
+        params.put("name", backupName);
+        params.put("commitName", commitName);
+        params.put("location", location);
+        BackupRestoreUtils.runCoreAdminCommand(replicaBaseUrl, coreName, CoreAdminAction.BACKUPCORE.toString(), params);
+      }
+
+      // Restore the backup
+      {
+        Map<String,String> params = new HashMap<>();
+        params.put("name", "snapshot." + backupName);
+        params.put("location", location);
+        BackupRestoreUtils.runCoreAdminCommand(replicaBaseUrl, coreName, CoreAdminAction.RESTORECORE.toString(), params);
+        BackupRestoreUtils.verifyDocs(nDocs, cluster.getSolrClient(), collectionName);
+      }
+
+      // Verify that the old index directory (before restore) contains only those index commits referred by snapshots.
+      {
+        List<IndexCommit> commits = listCommits(metaData.getIndexDirPath());
+        assertEquals(1, commits.size());
+        assertEquals(metaData.getGenerationNumber(), commits.get(0).getGeneration());
+      }
+
+      // Delete first snapshot
+      deleteSnapshot(adminClient, coreName, commitName);
+
+      // Verify that corresponding index files have NOT been deleted (due to reference counting).
+      assertFalse(listCommits(metaData.getIndexDirPath()).isEmpty());
+
+      // Delete second snapshot
+      deleteSnapshot(adminClient, coreName, duplicateCommit.getName());
+
+      // Verify that corresponding index files have been deleted.
+      assertTrue(listCommits(duplicateCommit.getIndexDirPath()).isEmpty());
+    }
+  }
+
+  @Test
+  public void testHandlingSharedIndexFiles() throws Exception {
+    CloudSolrClient solrClient = cluster.getSolrClient();
+    String collectionName = "SolrCoreSnapshots_IndexFileSharing";
+    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName, "conf1", 1, 1);
+    create.process(solrClient);
+
+    int nDocs = BackupRestoreUtils.indexDocs(cluster.getSolrClient(), collectionName, docsSeed);
+    DocCollection collectionState = solrClient.getZkStateReader().getClusterState().getCollection(collectionName);
+    assertEquals(1, collectionState.getActiveSlices().size());
+    Slice shard = collectionState.getActiveSlices().iterator().next();
+    assertEquals(1, shard.getReplicas().size());
+    Replica replica = shard.getReplicas().iterator().next();
+
+    String replicaBaseUrl = replica.getStr(BASE_URL_PROP);
+    String coreName = replica.getStr(ZkStateReader.CORE_NAME_PROP);
+    String backupName = TestUtil.randomSimpleString(random(), 1, 5);
+    String location = createTempDir().toFile().getAbsolutePath();
+
+    try (
+        SolrClient adminClient = getHttpSolrClient(cluster.getJettySolrRunners().get(0).getBaseUrl().toString());
+        SolrClient masterClient = getHttpSolrClient(replica.getCoreUrl())) {
+
+      int numTests = TestUtil.nextInt(random(), 2, 5);
+      List<SnapshotMetaData> snapshots = new ArrayList<>(numTests);
+
+      // Create multiple commits and create a snapshot per commit.
+      // This should result in Lucene reusing some of the segments for later index commits.
+      for (int attempt=0; attempt<numTests; attempt++) {
+        if (nDocs > 0) {
+          //Delete a few docs
+          int numDeletes = TestUtil.nextInt(random(), 1, nDocs);
+          for(int i=0; i<numDeletes; i++) {
+            masterClient.deleteByQuery("id:" + i);
+          }
+        }
+
+        // Add a few more
+        int moreAdds = TestUtil.nextInt(random(), 1, 100);
+        for (int i = 0; i < moreAdds; i++) {
+          SolrInputDocument doc = new SolrInputDocument();
+          doc.addField("id", i + nDocs);
+          doc.addField("name", "name = " + (i + nDocs));
+          masterClient.add(doc);
+        }
+        masterClient.commit();
+
+        // Create a snapshot
+        snapshots.add(createSnapshot(adminClient, coreName, "snapshot_" + attempt));
+      }
+
+      // Backup the earlier created snapshot.
+      {
+        Map<String,String> params = new HashMap<>();
+        params.put("name", backupName);
+        params.put("commitName", snapshots.get(0).getName());
+        params.put("location", location);
+        BackupRestoreUtils.runCoreAdminCommand(replicaBaseUrl, coreName, CoreAdminAction.BACKUPCORE.toString(), params);
+      }
+
+      // Restore the backup. The purpose of the restore operation is to change the *current* index directory.
+      // This is required since we delegate the file deletion to underlying IndexDeletionPolicy in case of
+      // *current* index directory. Hence for the purpose of this test, we want to ensure that the created
+      // snapshots are NOT in the *current* index directory.
+      {
+        Map<String,String> params = new HashMap<>();
+        params.put("name", "snapshot." + backupName);
+        params.put("location", location);
+        BackupRestoreUtils.runCoreAdminCommand(replicaBaseUrl, coreName, CoreAdminAction.RESTORECORE.toString(), params);
+      }
+
+      {
+        SnapshotMetaData snapshotMetaData = snapshots.get(0);
+
+        List<IndexCommit> commits = listCommits(snapshotMetaData.getIndexDirPath());
+        // Check if number of index commits are > 0 to ensure index file sharing.
+        assertTrue(commits.size() > 0);
+        Map<String,Integer> refCounts = SolrSnapshotManager.buildRefCounts(snapshots, commits);
+
+        Optional<IndexCommit> ic = commits.stream()
+            .filter(entry -> entry.getGeneration() == snapshotMetaData.getGenerationNumber())
+            .findFirst();
+        assertTrue(ic.isPresent());
+        Collection<String> nonSharedFiles = new ArrayList<>();
+        Collection<String> sharedFiles = new ArrayList<>();
+        for (String fileName : ic.get().getFileNames()) {
+          if (refCounts.getOrDefault(fileName, 0) > 1) {
+            sharedFiles.add(fileName);
+          } else {
+            nonSharedFiles.add(fileName);
+          }
+        }
+
+        // Delete snapshot
+        deleteSnapshot(adminClient, coreName, snapshotMetaData.getName());
+
+        // Verify that the shared files are not deleted.
+        for (String fileName : sharedFiles) {
+          Path path = Paths.get(snapshotMetaData.getIndexDirPath(), fileName);
+          assertTrue(path + " should exist.", Files.exists(path));
+        }
+
+        // Verify that the non-shared files are deleted.
+        for (String fileName : nonSharedFiles) {
+          Path path = Paths.get(snapshotMetaData.getIndexDirPath(), fileName);
+          assertFalse(path + " should not exist.", Files.exists(path));
+        }
+        }
+      }
+  }
+
+  @Test
+  public void testIndexOptimization() throws Exception {
+    CloudSolrClient solrClient = cluster.getSolrClient();
+    String collectionName = "SolrCoreSnapshots_IndexOptimization";
+    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName, "conf1", 1, 1);
+    create.process(solrClient);
+
+    int nDocs = BackupRestoreUtils.indexDocs(cluster.getSolrClient(), collectionName, docsSeed);
+
+    DocCollection collectionState = solrClient.getZkStateReader().getClusterState().getCollection(collectionName);
+    assertEquals(1, collectionState.getActiveSlices().size());
+    Slice shard = collectionState.getActiveSlices().iterator().next();
+    assertEquals(1, shard.getReplicas().size());
+    Replica replica = shard.getReplicas().iterator().next();
+
+    String coreName = replica.getStr(ZkStateReader.CORE_NAME_PROP);
+    String commitName = TestUtil.randomSimpleString(random(), 1, 5);
+
+    try (
+        SolrClient adminClient = getHttpSolrClient(cluster.getJettySolrRunners().get(0).getBaseUrl().toString());
+        SolrClient masterClient = getHttpSolrClient(replica.getCoreUrl())) {
+
+      SnapshotMetaData metaData = createSnapshot(adminClient, coreName, commitName);
+
+      int numTests = nDocs > 0 ? TestUtil.nextInt(random(), 1, 5) : 1;
+      for (int attempt=0; attempt<numTests; attempt++) {
+        //Modify existing index before we call optimize.
+        if (nDocs > 0) {
+          //Delete a few docs
+          int numDeletes = TestUtil.nextInt(random(), 1, nDocs);
+          for(int i=0; i<numDeletes; i++) {
+            masterClient.deleteByQuery("id:" + i);
+          }
+          //Add a few more
+          int moreAdds = TestUtil.nextInt(random(), 1, 100);
+          for (int i=0; i<moreAdds; i++) {
+            SolrInputDocument doc = new SolrInputDocument();
+            doc.addField("id", i + nDocs);
+            doc.addField("name", "name = " + (i + nDocs));
+            masterClient.add(doc);
+          }
+          masterClient.commit();
+        }
+      }
+
+      // Before invoking optimize command, verify that the index directory contains multiple commits (including the one we snapshotted earlier).
+      {
+        Collection<IndexCommit> commits = listCommits(metaData.getIndexDirPath());
+        // Verify that multiple index commits are stored in this directory.
+        assertTrue(commits.size() > 0);
+        // Verify that the snapshot commit is present in this directory.
+        assertTrue(commits.stream().filter(x -> x.getGeneration() == metaData.getGenerationNumber()).findFirst().isPresent());
+      }
+
+      // Optimize the index.
+      masterClient.optimize(true, true, 1);
+
+      // After invoking optimize command, verify that the index directory contains multiple commits (including the one we snapshotted earlier).
+      {
+        List<IndexCommit> commits = listCommits(metaData.getIndexDirPath());
+        // Verify that multiple index commits are stored in this directory.
+        assertTrue(commits.size() > 1);
+        // Verify that the snapshot commit is present in this directory.
+        assertTrue(commits.stream().filter(x -> x.getGeneration() == metaData.getGenerationNumber()).findFirst().isPresent());
+      }
+
+      // Delete the snapshot
+      deleteSnapshot(adminClient, coreName, metaData.getName());
+
+      // Add few documents. Without this the optimize command below does not take effect.
+      {
+        int moreAdds = TestUtil.nextInt(random(), 1, 100);
+        for (int i=0; i<moreAdds; i++) {
+          SolrInputDocument doc = new SolrInputDocument();
+          doc.addField("id", i + nDocs);
+          doc.addField("name", "name = " + (i + nDocs));
+          masterClient.add(doc);
+        }
+        masterClient.commit();
+      }
+
+      // Optimize the index.
+      masterClient.optimize(true, true, 1);
+
+      // Verify that the index directory contains only 1 index commit (which is not the same as the snapshotted commit).
+      Collection<IndexCommit> commits = listCommits(metaData.getIndexDirPath());
+      assertTrue(commits.size() == 1);
+      assertFalse(commits.stream().filter(x -> x.getGeneration() == metaData.getGenerationNumber()).findFirst().isPresent());
+    }
+  }
+
+  private SnapshotMetaData createSnapshot (SolrClient adminClient, String coreName, String commitName) throws Exception {
+    CreateSnapshot req = new CreateSnapshot(commitName);
+    req.setCoreName(coreName);
+    adminClient.request(req);
+
+    Collection<SnapshotMetaData> snapshots = listSnapshots(adminClient, coreName);
+    Optional<SnapshotMetaData> metaData = snapshots.stream().filter(x -> commitName.equals(x.getName())).findFirst();
+    assertTrue(metaData.isPresent());
+
+    return metaData.get();
+  }
+
+  private void deleteSnapshot(SolrClient adminClient, String coreName, String commitName) throws Exception {
+    DeleteSnapshot req = new DeleteSnapshot(commitName);
+    req.setCoreName(coreName);
+    adminClient.request(req);
+
+    Collection<SnapshotMetaData> snapshots = listSnapshots(adminClient, coreName);
+    assertFalse(snapshots.stream().filter(x -> commitName.equals(x.getName())).findFirst().isPresent());
+  }
+
+  private Collection<SnapshotMetaData> listSnapshots(SolrClient adminClient, String coreName) throws Exception {
+    ListSnapshots req = new ListSnapshots();
+    req.setCoreName(coreName);
+    NamedList resp = adminClient.request(req);
+    assertTrue( resp.get("snapshots") instanceof NamedList );
+    NamedList apiResult = (NamedList) resp.get("snapshots");
+
+    List<SnapshotMetaData> result = new ArrayList<>(apiResult.size());
+    for(int i = 0 ; i < apiResult.size(); i++) {
+      String commitName = apiResult.getName(i);
+      String indexDirPath = (String)((NamedList)apiResult.get(commitName)).get("indexDirPath");
+      long genNumber = Long.valueOf((String)((NamedList)apiResult.get(commitName)).get("generation"));
+      result.add(new SnapshotMetaData(commitName, indexDirPath, genNumber));
+    }
+    return result;
+  }
+
+  private List<IndexCommit> listCommits(String directory) throws Exception {
+    SimpleFSDirectory dir = new SimpleFSDirectory(Paths.get(directory));
+    try {
+      return DirectoryReader.listCommits(dir);
+    } catch (IndexNotFoundException ex) {
+      // This can happen when the delete snapshot functionality cleans up the index files (when the directory
+      // storing these files is not the *current* index directory).
+      return Collections.emptyList();
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b7aa0b56/solr/core/src/test/org/apache/solr/handler/BackupRestoreUtils.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/BackupRestoreUtils.java b/solr/core/src/test/org/apache/solr/handler/BackupRestoreUtils.java
index e2f4304..34509cf 100644
--- a/solr/core/src/test/org/apache/solr/handler/BackupRestoreUtils.java
+++ b/solr/core/src/test/org/apache/solr/handler/BackupRestoreUtils.java
@@ -18,11 +18,15 @@
 package org.apache.solr.handler;
 
 import java.io.IOException;
+import java.io.InputStream;
 import java.lang.invoke.MethodHandles;
+import java.net.URL;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 import java.util.Random;
 
+import org.apache.commons.io.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 import org.apache.solr.client.solrj.SolrClient;
@@ -64,4 +68,37 @@ public class BackupRestoreUtils extends LuceneTestCase {
     assertEquals(0, response.getStatus());
     assertEquals(nDocs, response.getResults().getNumFound());
   }
+
+  public static void runCoreAdminCommand(String baseUrl, String coreName, String action, Map<String,String> params) throws IOException {
+    StringBuilder builder = new StringBuilder();
+    builder.append(baseUrl);
+    builder.append("/admin/cores?action=");
+    builder.append(action);
+    builder.append("&core=");
+    builder.append(coreName);
+    for (Map.Entry<String,String> p : params.entrySet()) {
+      builder.append("&");
+      builder.append(p.getKey());
+      builder.append("=");
+      builder.append(p.getValue());
+    }
+    String masterUrl = builder.toString();
+    executeHttpRequest(masterUrl);
+  }
+
+  public static void runReplicationHandlerCommand(String baseUrl, String coreName, String action, String repoName, String backupName) throws IOException {
+    String masterUrl = baseUrl + "/" + coreName + ReplicationHandler.PATH + "?command=" + action + "&repository="+repoName+"&name="+backupName;
+    executeHttpRequest(masterUrl);
+  }
+
+  static void executeHttpRequest(String requestUrl) throws IOException {
+    InputStream stream = null;
+    try {
+      URL url = new URL(requestUrl);
+      stream = url.openStream();
+      stream.close();
+    } finally {
+      IOUtils.closeQuietly(stream);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b7aa0b56/solr/core/src/test/org/apache/solr/handler/TestHdfsBackupRestoreCore.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/TestHdfsBackupRestoreCore.java b/solr/core/src/test/org/apache/solr/handler/TestHdfsBackupRestoreCore.java
index a840428..4e8d4cc 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestHdfsBackupRestoreCore.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestHdfsBackupRestoreCore.java
@@ -18,11 +18,11 @@
 package org.apache.solr.handler;
 
 import java.io.IOException;
-import java.io.InputStream;
 import java.lang.invoke.MethodHandles;
 import java.net.URI;
 import java.net.URISyntaxException;
-import java.net.URL;
+import java.util.HashMap;
+import java.util.Map;
 
 import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters;
 import org.apache.commons.io.IOUtils;
@@ -44,6 +44,7 @@ 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.params.CoreAdminParams.CoreAdminAction;
 import org.apache.solr.util.BadHdfsThreadsFilter;
 import org.junit.AfterClass;
@@ -176,16 +177,19 @@ public class TestHdfsBackupRestoreCore extends SolrCloudTestCase {
     try (SolrClient masterClient = getHttpSolrClient(replicaBaseUrl)) {
       // Create a backup.
       if (testViaReplicationHandler) {
-        log.info("Running Backup/restore via replication handler");
-        runReplicationHandlerCommand(baseUrl, coreName, ReplicationHandler.CMD_BACKUP, "hdfs", backupName);
+        log.info("Running Backup via replication handler");
+        BackupRestoreUtils.runReplicationHandlerCommand(baseUrl, coreName, ReplicationHandler.CMD_BACKUP, "hdfs", backupName);
         CheckBackupStatus checkBackupStatus = new CheckBackupStatus((HttpSolrClient) masterClient, coreName, null);
         while (!checkBackupStatus.success) {
           checkBackupStatus.fetchStatus();
           Thread.sleep(1000);
         }
       } else {
-        log.info("Running Backup/restore via core admin api");
-        runCoreAdminCommand(replicaBaseUrl, coreName, CoreAdminAction.BACKUPCORE.toString(), "hdfs", backupName);
+        log.info("Running Backup via core admin api");
+        Map<String,String> params = new HashMap<>();
+        params.put("name", backupName);
+        params.put(CoreAdminParams.BACKUP_REPOSITORY, "hdfs");
+        BackupRestoreUtils.runCoreAdminCommand(replicaBaseUrl, coreName, CoreAdminAction.BACKUPCORE.toString(), params);
       }
 
       int numRestoreTests = nDocs > 0 ? TestUtil.nextInt(random(), 1, 5) : 1;
@@ -214,38 +218,22 @@ public class TestHdfsBackupRestoreCore extends SolrCloudTestCase {
         }
         // Snapshooter prefixes "snapshot." to the backup name.
         if (testViaReplicationHandler) {
+          log.info("Running Restore via replication handler");
           // Snapshooter prefixes "snapshot." to the backup name.
-          runReplicationHandlerCommand(baseUrl, coreName, ReplicationHandler.CMD_RESTORE, "hdfs", backupName);
+          BackupRestoreUtils.runReplicationHandlerCommand(baseUrl, coreName, ReplicationHandler.CMD_RESTORE, "hdfs", backupName);
           while (!TestRestoreCore.fetchRestoreStatus(baseUrl, coreName)) {
             Thread.sleep(1000);
           }
         } else {
-          runCoreAdminCommand(replicaBaseUrl, coreName, CoreAdminAction.RESTORECORE.toString(), "hdfs", "snapshot." + backupName);
+          log.info("Running Restore via core admin api");
+          Map<String,String> params = new HashMap<>();
+          params.put("name", "snapshot." + backupName);
+          params.put(CoreAdminParams.BACKUP_REPOSITORY, "hdfs");
+          BackupRestoreUtils.runCoreAdminCommand(replicaBaseUrl, coreName, CoreAdminAction.RESTORECORE.toString(), params);
         }
         //See if restore was successful by checking if all the docs are present again
         BackupRestoreUtils.verifyDocs(nDocs, masterClient, coreName);
       }
     }
   }
-
-  static void runCoreAdminCommand(String baseUrl, String coreName, String action, String repoName, String backupName) throws IOException {
-    String masterUrl = baseUrl + "/admin/cores?action=" + action + "&core="+coreName+"&repository="+repoName+"&name="+backupName;
-    executeHttpRequest(masterUrl);
-  }
-
-  static void runReplicationHandlerCommand(String baseUrl, String coreName, String action, String repoName, String backupName) throws IOException {
-    String masterUrl = baseUrl + "/" + coreName + ReplicationHandler.PATH + "?command=" + action + "&repository="+repoName+"&name="+backupName;
-    executeHttpRequest(masterUrl);
-  }
-
-  static void executeHttpRequest(String requestUrl) throws IOException {
-    InputStream stream = null;
-    try {
-      URL url = new URL(requestUrl);
-      stream = url.openStream();
-      stream.close();
-    } finally {
-      IOUtils.closeQuietly(stream);
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b7aa0b56/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 b9bcf7b..08c462b 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
@@ -20,6 +20,7 @@ import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
+import java.io.FilenameFilter;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
@@ -66,6 +67,7 @@ import org.apache.solr.core.CachingDirectoryFactory;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.StandardDirectoryFactory;
+import org.apache.solr.core.snapshots.SolrSnapshotMetaDataManager;
 import org.apache.solr.util.FileUtils;
 import org.junit.After;
 import org.junit.Before;
@@ -896,8 +898,8 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
       CachingDirectoryFactory dirFactory = (CachingDirectoryFactory) core.getDirectoryFactory();
       synchronized (dirFactory) {
         Set<String> livePaths = dirFactory.getLivePaths();
-        // one for data, one for hte index under data
-        assertEquals(livePaths.toString(), 2, livePaths.size());
+        // one for data, one for hte index under data and one for the snapshot metadata.
+        assertEquals(livePaths.toString(), 3, livePaths.size());
         // :TODO: assert that one of the paths is a subpath of hte other
       }
       if (dirFactory instanceof StandardDirectoryFactory) {
@@ -908,14 +910,14 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
   }
 
   private int indexDirCount(String ddir) {
-    String[] list = new File(ddir).list();
-    int cnt = 0;
-    for (String file : list) {
-      if (!file.endsWith(".properties")) {
-        cnt++;
+    String[] list = new File(ddir).list(new FilenameFilter() {
+      @Override
+      public boolean accept(File dir, String name) {
+        File f = new File(dir, name);
+        return f.isDirectory() && !SolrSnapshotMetaDataManager.SNAPSHOT_METADATA_DIR.equals(name);
       }
-    }
-    return cnt;
+    });
+    return list.length;
   }
 
   private void pullFromMasterToSlave() throws MalformedURLException,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b7aa0b56/solr/solrj/src/java/org/apache/solr/client/solrj/request/CoreAdminRequest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CoreAdminRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CoreAdminRequest.java
index 7d9e356..f3e4e19 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CoreAdminRequest.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CoreAdminRequest.java
@@ -453,6 +453,63 @@ public class CoreAdminRequest extends SolrRequest<CoreAdminResponse> {
 
   }
 
+  public static class CreateSnapshot extends CoreAdminRequest {
+    private String commitName;
+
+    public CreateSnapshot(String commitName) {
+      super();
+      this.action = CoreAdminAction.CREATESNAPSHOT;
+      if(commitName == null) {
+        throw new NullPointerException("Please specify non null value for commitName parameter.");
+      }
+      this.commitName = commitName;
+    }
+
+    public String getCommitName() {
+      return commitName;
+    }
+
+    @Override
+    public SolrParams getParams() {
+      ModifiableSolrParams params = new ModifiableSolrParams(super.getParams());
+      params.set(CoreAdminParams.COMMIT_NAME, this.commitName);
+      return params;
+    }
+  }
+
+  public static class DeleteSnapshot extends CoreAdminRequest {
+    private String commitName;
+
+    public DeleteSnapshot(String commitName) {
+      super();
+      this.action = CoreAdminAction.DELETESNAPSHOT;
+
+      if(commitName == null) {
+        throw new NullPointerException("Please specify non null value for commitName parameter.");
+      }
+      this.commitName = commitName;
+    }
+
+    public String getCommitName() {
+      return commitName;
+    }
+
+    @Override
+    public SolrParams getParams() {
+      ModifiableSolrParams params = new ModifiableSolrParams(super.getParams());
+      params.set(CoreAdminParams.COMMIT_NAME, this.commitName);
+      return params;
+    }
+  }
+
+  public static class ListSnapshots extends CoreAdminRequest {
+    public ListSnapshots() {
+      super();
+      this.action = CoreAdminAction.LISTSNAPSHOTS;
+    }
+  }
+
+
   public CoreAdminRequest()
   {
     super( METHOD.GET, "/admin/cores" );

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b7aa0b56/solr/solrj/src/java/org/apache/solr/common/params/CoreAdminParams.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/params/CoreAdminParams.java b/solr/solrj/src/java/org/apache/solr/common/params/CoreAdminParams.java
index 7455cbf..7f90a90 100644
--- a/solr/solrj/src/java/org/apache/solr/common/params/CoreAdminParams.java
+++ b/solr/solrj/src/java/org/apache/solr/common/params/CoreAdminParams.java
@@ -118,6 +118,11 @@ public abstract class CoreAdminParams
    */
   public static final String BACKUP_LOCATION = "location";
 
+  /**
+   * A parameter to specify the name of the commit to be stored during the backup operation.
+   */
+  public static final String COMMIT_NAME = "commitName";
+
   public enum CoreAdminAction {
     STATUS(true),
     UNLOAD,
@@ -141,7 +146,10 @@ public abstract class CoreAdminParams
     INVOKE,
     //Internal APIs to backup and restore a core
     BACKUPCORE,
-    RESTORECORE;
+    RESTORECORE,
+    CREATESNAPSHOT,
+    DELETESNAPSHOT,
+    LISTSNAPSHOTS;
 
     public final boolean isRead;
 


[08/40] lucene-solr:apiv2: SOLR-9076: disable broken nightly tests MorphlineBasicMiniMRTest and MorphlineGoLiveMiniMRTest via @AwaitsFix

Posted by sa...@apache.org.
SOLR-9076: disable broken nightly tests MorphlineBasicMiniMRTest and MorphlineGoLiveMiniMRTest via @AwaitsFix


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

Branch: refs/heads/apiv2
Commit: 85a585c51698edd823769a159856524407cf6456
Parents: a00e344
Author: Steve Rowe <sa...@apache.org>
Authored: Fri Jul 22 14:26:32 2016 -0400
Committer: Steve Rowe <sa...@apache.org>
Committed: Fri Jul 22 14:26:32 2016 -0400

----------------------------------------------------------------------
 .../src/test/org/apache/solr/hadoop/MorphlineBasicMiniMRTest.java  | 2 ++
 .../src/test/org/apache/solr/hadoop/MorphlineGoLiveMiniMRTest.java | 2 ++
 2 files changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/85a585c5/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineBasicMiniMRTest.java
----------------------------------------------------------------------
diff --git a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineBasicMiniMRTest.java b/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineBasicMiniMRTest.java
index 8d39906..b32f112 100644
--- a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineBasicMiniMRTest.java
+++ b/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineBasicMiniMRTest.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.util.JarFinder;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.lucene.util.Constants;
+import org.apache.lucene.util.LuceneTestCase.AwaitsFix;
 import org.apache.lucene.util.LuceneTestCase.Slow;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.cloud.AbstractZkTestCase;
@@ -67,6 +68,7 @@ import com.carrotsearch.randomizedtesting.annotations.ThreadLeakZombies.Conseque
 })
 @Slow
 @Nightly
+@AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/SOLR-9076")
 public class MorphlineBasicMiniMRTest extends SolrTestCaseJ4 {
   
   private static final boolean ENABLE_LOCAL_JOB_RUNNER = false; // for debugging only

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/85a585c5/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineGoLiveMiniMRTest.java
----------------------------------------------------------------------
diff --git a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineGoLiveMiniMRTest.java b/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineGoLiveMiniMRTest.java
index ce3d8c4..adc8d88 100644
--- a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineGoLiveMiniMRTest.java
+++ b/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineGoLiveMiniMRTest.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.util.ToolRunner;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.lucene.util.Constants;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.LuceneTestCase.AwaitsFix;
 import org.apache.lucene.util.LuceneTestCase.Slow;
 import org.apache.solr.SolrTestCaseJ4.SuppressSSL;
 import org.apache.solr.client.solrj.SolrClient;
@@ -97,6 +98,7 @@ import com.carrotsearch.randomizedtesting.annotations.ThreadLeakZombies.Conseque
 @SuppressSSL // SSL does not work with this test for currently unknown reasons
 @Slow
 @Nightly
+@AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/SOLR-9076")
 public class MorphlineGoLiveMiniMRTest extends AbstractFullDistribZkTestBase {
   
   private static final int RECORD_COUNT = 2104;


[38/40] lucene-solr:apiv2: LUCENE-7400 - splitter should be able to group using sorted set dv

Posted by sa...@apache.org.
LUCENE-7400 - splitter should be able to group using sorted set dv


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

Branch: refs/heads/apiv2
Commit: 53a34b312e78ce6f56c0bb41304ac834b28b9534
Parents: 9554719
Author: Tommaso Teofili <to...@apache.org>
Authored: Sat Jul 30 07:58:02 2016 +0200
Committer: Tommaso Teofili <to...@apache.org>
Committed: Sat Jul 30 07:58:25 2016 +0200

----------------------------------------------------------------------
 .../lucene/classification/utils/DatasetSplitter.java  | 14 ++++++++++++--
 1 file changed, 12 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/53a34b31/lucene/classification/src/java/org/apache/lucene/classification/utils/DatasetSplitter.java
----------------------------------------------------------------------
diff --git a/lucene/classification/src/java/org/apache/lucene/classification/utils/DatasetSplitter.java b/lucene/classification/src/java/org/apache/lucene/classification/utils/DatasetSplitter.java
index 374624b..8bb0b1d 100644
--- a/lucene/classification/src/java/org/apache/lucene/classification/utils/DatasetSplitter.java
+++ b/lucene/classification/src/java/org/apache/lucene/classification/utils/DatasetSplitter.java
@@ -30,6 +30,7 @@ import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.ScoreDoc;
@@ -82,11 +83,20 @@ public class DatasetSplitter {
     // get the exact no. of existing classes
     int noOfClasses = 0;
     for (LeafReaderContext leave : originalIndex.leaves()) {
+      long valueCount = 0;
       SortedDocValues classValues = leave.reader().getSortedDocValues(classFieldName);
+      if (classValues != null) {
+        valueCount = classValues.getValueCount();
+      } else {
+        SortedSetDocValues sortedSetDocValues = leave.reader().getSortedSetDocValues(classFieldName);
+        if (sortedSetDocValues != null) {
+          valueCount = sortedSetDocValues.getValueCount();
+        }
+      }
       if (classValues == null) {
-        throw new IllegalStateException("the classFieldName \"" + classFieldName + "\" must index sorted doc values");
+        throw new IllegalStateException("field \"" + classFieldName + "\" must have sorted (set) doc values");
       }
-      noOfClasses += classValues.getValueCount();
+      noOfClasses += valueCount;
     }
 
     try {


[03/40] lucene-solr:apiv2: Merge remote-tracking branch 'origin/master'

Posted by sa...@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/f8a99dc3
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/f8a99dc3
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/f8a99dc3

Branch: refs/heads/apiv2
Commit: f8a99dc3c3918f71ffc9e4591fcaf2d79af2874a
Parents: 941dff9 f5964ec
Author: Noble Paul <no...@apache.org>
Authored: Thu Jul 21 20:26:31 2016 +0530
Committer: Noble Paul <no...@apache.org>
Committed: Thu Jul 21 20:26:31 2016 +0530

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   3 +
 .../lucene/document/DoubleRangeField.java       | 262 ++++++++++++
 .../apache/lucene/document/RangeFieldQuery.java | 313 ++++++++++++++
 .../lucene/document/TestDoubleRangeField.java   |  67 +++
 .../search/BaseRangeFieldQueryTestCase.java     | 403 +++++++++++++++++++
 .../search/TestDoubleRangeFieldQueries.java     | 106 +++++
 6 files changed, 1154 insertions(+)
----------------------------------------------------------------------



[17/40] lucene-solr:apiv2: SOLR-9339: NPE in CloudSolrClient when the response is null

Posted by sa...@apache.org.
SOLR-9339: NPE in CloudSolrClient when the response is null


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

Branch: refs/heads/apiv2
Commit: 9a24dc5d2ca890a50dc9751cb3abfd93c9939fee
Parents: 4ed68bc
Author: Noble Paul <no...@apache.org>
Authored: Tue Jul 26 13:08:28 2016 +0530
Committer: Noble Paul <no...@apache.org>
Committed: Tue Jul 26 13:08:28 2016 +0530

----------------------------------------------------------------------
 solr/CHANGES.txt                                                   | 2 ++
 .../java/org/apache/solr/client/solrj/impl/CloudSolrClient.java    | 2 +-
 2 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9a24dc5d/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index e392df3..4da2d0e 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -163,6 +163,8 @@ Bug Fixes
 
 * SOLR-9334: CloudSolrClient.collectionStateCache is unbounded (noble)
 
+* SOLR-9339: NPE in CloudSolrClient when the response is null (noble)
+
 
 Optimizations
 ----------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9a24dc5d/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
index b4c9b4f..1f1c675 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
@@ -1060,7 +1060,7 @@ public class CloudSolrClient extends SolrClient {
     try {
       resp = sendRequest(request, collection);
       //to avoid an O(n) operation we always add STATE_VERSION to the last and try to read it from there
-      Object o = resp.get(STATE_VERSION, resp.size()-1);
+      Object o = resp == null || resp.size() == 0 ? null : resp.get(STATE_VERSION, resp.size() - 1);
       if(o != null && o instanceof Map) {
         //remove this because no one else needs this and tests would fail if they are comparing responses
         resp.remove(resp.size()-1);


[20/40] lucene-solr:apiv2: SOLR-9340: Change ZooKeeper disconnect and session expiry related logging from INFO to WARN to make debugging easier

Posted by sa...@apache.org.
SOLR-9340: Change ZooKeeper disconnect and session expiry related logging from INFO to WARN to make debugging easier


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

Branch: refs/heads/apiv2
Commit: 3603a6288980adef028fccf1e80048551d017f52
Parents: f906862
Author: Varun Thacker <va...@apache.org>
Authored: Tue Jul 26 23:41:39 2016 +0530
Committer: Varun Thacker <va...@apache.org>
Committed: Tue Jul 26 23:41:39 2016 +0530

----------------------------------------------------------------------
 solr/CHANGES.txt                                     |  3 +++
 .../apache/solr/common/cloud/ConnectionManager.java  | 15 +++++++++++----
 .../solr/common/cloud/DefaultConnectionStrategy.java |  4 ++--
 3 files changed, 16 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3603a628/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 64c4291..eab3780 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -200,6 +200,9 @@ Other Changes
 * SOLR-9163: Sync up basic_configs and data_driven_schema_configs, removing almost all differences
   except what is required for schemaless. (yonik)
 
+* SOLR-9340: Change ZooKeeper disconnect and session expiry related logging from INFO to WARN to
+  make debugging easier (Varun Thacker)
+
 ==================  6.1.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/3603a628/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java
index 1754a2d..c944dec 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java
@@ -28,6 +28,10 @@ import org.apache.zookeeper.Watcher.Event.KeeperState;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.zookeeper.Watcher.Event.KeeperState.AuthFailed;
+import static org.apache.zookeeper.Watcher.Event.KeeperState.Disconnected;
+import static org.apache.zookeeper.Watcher.Event.KeeperState.Expired;
+
 public class ConnectionManager implements Watcher {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
@@ -100,7 +104,10 @@ public class ConnectionManager implements Watcher {
 
   @Override
   public void process(WatchedEvent event) {
-    if (log.isInfoEnabled()) {
+    if (event.getState() == AuthFailed || event.getState() == Disconnected || event.getState() == Expired) {
+      log.warn("Watcher " + this + " name:" + name + " got event " + event
+          + " path:" + event.getPath() + " type:" + event.getType());
+    } else if (log.isInfoEnabled()) {
       log.info("Watcher " + this + " name:" + name + " got event " + event
           + " path:" + event.getPath() + " type:" + event.getType());
     }
@@ -115,12 +122,12 @@ public class ConnectionManager implements Watcher {
     if (state == KeeperState.SyncConnected) {
       connected();
       connectionStrategy.connected();
-    } else if (state == KeeperState.Expired) {
+    } else if (state == Expired) {
       // we don't call disconnected here, because we know we are expired
       connected = false;
       likelyExpiredState = LikelyExpiredState.EXPIRED;
       
-      log.info("Our previous ZooKeeper session was expired. Attempting to reconnect to recover relationship with ZooKeeper...");
+      log.warn("Our previous ZooKeeper session was expired. Attempting to reconnect to recover relationship with ZooKeeper...");
       
       if (beforeReconnect != null) {
         try {
@@ -176,7 +183,7 @@ public class ConnectionManager implements Watcher {
       } while (!isClosed);
       log.info("Connected:" + connected);
     } else if (state == KeeperState.Disconnected) {
-      log.info("zkClient has disconnected");
+      log.warn("zkClient has disconnected");
       disconnected();
       connectionStrategy.disconnected();
     } else if (state == KeeperState.AuthFailed) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3603a628/solr/solrj/src/java/org/apache/solr/common/cloud/DefaultConnectionStrategy.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/DefaultConnectionStrategy.java b/solr/solrj/src/java/org/apache/solr/common/cloud/DefaultConnectionStrategy.java
index cbec324..e16ca68 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/DefaultConnectionStrategy.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/DefaultConnectionStrategy.java
@@ -49,7 +49,7 @@ public class DefaultConnectionStrategy extends ZkClientConnectionStrategy {
   @Override
   public void reconnect(final String serverAddress, final int zkClientTimeout,
       final Watcher watcher, final ZkUpdate updater) throws IOException {
-    log.info("Connection expired - starting a new one...");
+    log.warn("Connection expired - starting a new one...");
     SolrZooKeeper zk = createSolrZooKeeper(serverAddress, zkClientTimeout, watcher);
     boolean success = false;
     try {
@@ -59,7 +59,7 @@ public class DefaultConnectionStrategy extends ZkClientConnectionStrategy {
       log.info("Reconnected to ZooKeeper");
     } catch (Exception e) {
       SolrException.log(log, "Reconnect to ZooKeeper failed", e);
-      log.info("Reconnect to ZooKeeper failed");
+      log.warn("Reconnect to ZooKeeper failed");
     } finally {
       if (!success) {
         try {


[09/40] lucene-solr:apiv2: LUCENE-7383: Fix 'dimensionNumBytes' validation

Posted by sa...@apache.org.
LUCENE-7383: Fix 'dimensionNumBytes' validation


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

Branch: refs/heads/apiv2
Commit: 9b85f6827892d6ff3d53103f1cc2bc43b0d0a132
Parents: 85a585c
Author: Martijn van Groningen <mv...@apache.org>
Authored: Fri Jul 22 16:26:05 2016 +0200
Committer: Martijn van Groningen <mv...@apache.org>
Committed: Mon Jul 25 08:25:51 2016 +0200

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


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9b85f682/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 6d66355..1edac7e 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -76,6 +76,9 @@ Bug Fixes
   wrong default AttributeFactory for new Tokenizers.
   (Terry Smith, Uwe Schindler)
 
+* LUCENE-7389: Fix FieldType.setDimensions(...) validation for the dimensionNumBytes
+  parameter. (Martijn van Groningen)
+
 Improvements
 
 * LUCENE-7323: Compound file writing now verifies the incoming

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9b85f682/lucene/core/src/java/org/apache/lucene/document/FieldType.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/FieldType.java b/lucene/core/src/java/org/apache/lucene/document/FieldType.java
index 2c0a62c..e0f058f 100644
--- a/lucene/core/src/java/org/apache/lucene/document/FieldType.java
+++ b/lucene/core/src/java/org/apache/lucene/document/FieldType.java
@@ -374,7 +374,7 @@ public class FieldType implements IndexableFieldType  {
     if (dimensionNumBytes < 0) {
       throw new IllegalArgumentException("dimensionNumBytes must be >= 0; got " + dimensionNumBytes);
     }
-    if (dimensionCount > PointValues.MAX_NUM_BYTES) {
+    if (dimensionNumBytes > PointValues.MAX_NUM_BYTES) {
       throw new IllegalArgumentException("dimensionNumBytes must be <= " + PointValues.MAX_NUM_BYTES + "; got " + dimensionNumBytes);
     }
     if (dimensionCount == 0) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9b85f682/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java b/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java
index 9693c5c..a846c27 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java
@@ -362,9 +362,8 @@ public class TestPointValues extends LuceneTestCase {
     IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
     IndexWriter w = new IndexWriter(dir, iwc);
     Document doc = new Document();
-    doc.add(new BinaryPoint("dim", new byte[PointValues.MAX_NUM_BYTES+1]));
     expectThrows(IllegalArgumentException.class, () -> {
-      w.addDocument(doc);
+      doc.add(new BinaryPoint("dim", new byte[PointValues.MAX_NUM_BYTES+1]));
     });
 
     Document doc2 = new Document();


[33/40] lucene-solr:apiv2: SOLR-9279: new function queries: gt, gte, lt, lte, eq Lucene Queries module: new ComparisonBoolFunction base class

Posted by sa...@apache.org.
SOLR-9279: new function queries: gt, gte, lt, lte, eq
Lucene Queries module: new ComparisonBoolFunction base class


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

Branch: refs/heads/apiv2
Commit: d12b93e2729036b0c04621114429c25739499243
Parents: cead204
Author: David Smiley <ds...@apache.org>
Authored: Thu Jul 28 22:45:43 2016 -0400
Committer: David Smiley <ds...@apache.org>
Committed: Thu Jul 28 22:45:43 2016 -0400

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   3 +
 .../valuesource/ComparisonBoolFunction.java     | 105 +++++++++++++++++++
 solr/CHANGES.txt                                |   3 +
 .../apache/solr/search/ValueSourceParser.java   |  52 +++++++++
 .../function/SolrComparisonBoolFunction.java    |  58 ++++++++++
 .../apache/solr/search/QueryEqualityTest.java   |  27 ++++-
 .../solr/search/function/TestFunctionQuery.java |  69 +++++++++++-
 7 files changed, 311 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d12b93e2/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 6958660..a685839 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -65,6 +65,9 @@ New Features
   Polygon instances from a standard GeoJSON string (Robert Muir, Mike
   McCandless)
 
+* SOLR-9279: Queries module: new ComparisonBoolFunction base class
+  (Doug Turnbull via David Smiley)
+
 Bug Fixes
 
 * LUCENE-6662: Fixed potential resource leaks. (Rishabh Patel via Adrien Grand)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d12b93e2/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/ComparisonBoolFunction.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/ComparisonBoolFunction.java b/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/ComparisonBoolFunction.java
new file mode 100644
index 0000000..82d723a
--- /dev/null
+++ b/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/ComparisonBoolFunction.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.queries.function.valuesource;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.queries.function.docvalues.BoolDocValues;
+import org.apache.lucene.search.IndexSearcher;
+
+/**
+ * Base class for comparison operators useful within an "if"/conditional.
+ */
+public abstract class ComparisonBoolFunction extends BoolFunction {
+
+  private final ValueSource lhs;
+  private final ValueSource rhs;
+  private final String name;
+
+  public ComparisonBoolFunction(ValueSource lhs, ValueSource rhs, String name) {
+    this.lhs = lhs;
+    this.rhs = rhs;
+    this.name = name;
+  }
+
+  /** Perform the comparison, returning true or false */
+  public abstract boolean compare(int doc, FunctionValues lhs, FunctionValues rhs);
+
+  /** Uniquely identify the operation (ie "gt", "lt" "gte", etc) */
+  public String name() {
+    return this.name;
+  }
+
+  @Override
+  public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
+    final FunctionValues lhsVal = this.lhs.getValues(context, readerContext);
+    final FunctionValues rhsVal = this.rhs.getValues(context, readerContext);
+    final String compLabel = this.name();
+
+    return new BoolDocValues(this) {
+      @Override
+      public boolean boolVal(int doc) {
+        return compare(doc, lhsVal, rhsVal);
+      }
+
+      @Override
+      public String toString(int doc) {
+        return compLabel + "(" + lhsVal.toString(doc) + "," + rhsVal.toString(doc) + ")";
+      }
+
+      @Override
+      public boolean exists(int doc) {
+        return lhsVal.exists(doc) && rhsVal.exists(doc);
+      }
+
+    };
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this.getClass() != o.getClass()) return false;
+    ComparisonBoolFunction other = (ComparisonBoolFunction)o;
+    return name().equals(other.name())
+        && lhs.equals(other.lhs)
+        && rhs.equals(other.rhs);  }
+
+  @Override
+  public int hashCode() {
+    int h = this.getClass().hashCode();
+    h = h * 31 + this.name().hashCode();
+    h = h * 31 + lhs.hashCode();
+    h = h * 31 + rhs.hashCode();
+    return h;
+  }
+
+  @Override
+  public String description() {
+      return name() + "(" + lhs.description() + "," + rhs.description() + ")";
+  }
+
+  @Override
+  public void createWeight(Map context, IndexSearcher searcher) throws IOException {
+    lhs.createWeight(context, searcher);
+    rhs.createWeight(context, searcher);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d12b93e2/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 22d4e99..58743b1 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -108,6 +108,9 @@ New Features
   doing a core backup, and in replication.  Snapshot metadata is stored in a new snapshot_metadata/ dir.
   (Hrishikesh Gadre via David Smiley)
 
+* SOLR-9279: New boolean comparison function queries comparing numeric arguments: gt, gte, lt, lte, eq
+  (Doug Turnbull, David Smiley)
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d12b93e2/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java b/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
index 65a4d0d..dd0db96 100644
--- a/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
+++ b/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
@@ -64,6 +64,7 @@ import org.apache.solr.search.facet.UniqueAgg;
 import org.apache.solr.search.function.CollapseScoreFunction;
 import org.apache.solr.search.function.OrdFieldSource;
 import org.apache.solr.search.function.ReverseOrdFieldSource;
+import org.apache.solr.search.function.SolrComparisonBoolFunction;
 import org.apache.solr.search.function.distance.GeoDistValueSourceParser;
 import org.apache.solr.search.function.distance.GeohashFunction;
 import org.apache.solr.search.function.distance.GeohashHaversineFunction;
@@ -822,6 +823,57 @@ public abstract class ValueSourceParser implements NamedListInitializedPlugin {
       }
     });
 
+    addParser("gt", new ValueSourceParser() {
+      @Override
+      public ValueSource parse(FunctionQParser fp) throws SyntaxError {
+        ValueSource lhsValSource = fp.parseValueSource();
+        ValueSource rhsValSource = fp.parseValueSource();
+
+        return new SolrComparisonBoolFunction(lhsValSource, rhsValSource, "gt", (cmp) -> cmp > 0);
+      }
+    });
+
+    addParser("lt", new ValueSourceParser() {
+      @Override
+      public ValueSource parse(FunctionQParser fp) throws SyntaxError {
+        ValueSource lhsValSource = fp.parseValueSource();
+        ValueSource rhsValSource = fp.parseValueSource();
+
+        return new SolrComparisonBoolFunction(lhsValSource, rhsValSource, "lt", (cmp) -> cmp < 0);
+      }
+    });
+
+    addParser("gte", new ValueSourceParser() {
+      @Override
+      public ValueSource parse(FunctionQParser fp) throws SyntaxError {
+        ValueSource lhsValSource = fp.parseValueSource();
+        ValueSource rhsValSource = fp.parseValueSource();
+
+        return new SolrComparisonBoolFunction(lhsValSource, rhsValSource, "gte", (cmp) -> cmp >= 0);
+
+      }
+    });
+
+    addParser("lte", new ValueSourceParser() {
+      @Override
+      public ValueSource parse(FunctionQParser fp) throws SyntaxError {
+        ValueSource lhsValSource = fp.parseValueSource();
+        ValueSource rhsValSource = fp.parseValueSource();
+
+        return new SolrComparisonBoolFunction(lhsValSource, rhsValSource, "lte", (cmp) -> cmp <= 0);
+      }
+    });
+
+    addParser("eq", new ValueSourceParser() {
+      @Override
+      public ValueSource parse(FunctionQParser fp) throws SyntaxError {
+        ValueSource lhsValSource = fp.parseValueSource();
+        ValueSource rhsValSource = fp.parseValueSource();
+
+        return new SolrComparisonBoolFunction(lhsValSource, rhsValSource, "eq", (cmp) -> cmp == 0);
+      }
+    });
+
     addParser("def", new ValueSourceParser() {
       @Override
       public ValueSource parse(FunctionQParser fp) throws SyntaxError {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d12b93e2/solr/core/src/java/org/apache/solr/search/function/SolrComparisonBoolFunction.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/function/SolrComparisonBoolFunction.java b/solr/core/src/java/org/apache/solr/search/function/SolrComparisonBoolFunction.java
new file mode 100644
index 0000000..c994fbb
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/search/function/SolrComparisonBoolFunction.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.search.function;
+
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.queries.function.docvalues.IntDocValues;
+import org.apache.lucene.queries.function.docvalues.LongDocValues;
+import org.apache.lucene.queries.function.valuesource.ComparisonBoolFunction;
+
+/**
+ * Refines {@link ComparisonBoolFunction} to compare based on a 'long' or 'double' depending on if the
+ * any of the FunctionValues are {@link LongDocValues}.
+ */
+public class SolrComparisonBoolFunction extends ComparisonBoolFunction {
+
+  private final Compare cmp;
+
+  public interface Compare {
+    boolean compare(int integer);
+  }
+
+  public SolrComparisonBoolFunction(ValueSource lhs, ValueSource rhs, String name, Compare cmp) {
+    super(lhs, rhs, name);
+    this.cmp = cmp;
+  }
+
+  @Override
+  public boolean compare(int doc, FunctionValues lhs, FunctionValues rhs) {
+    // TODO consider a separate FunctionValues impl, one for Long, one for Double
+    // performs the safest possible numeric comparison, if both lhs and rhs are Longs, then
+    // we perform a Long comparison to avoid the issues with precision when casting to doubles
+    boolean lhsAnInt = (lhs instanceof LongDocValues || lhs instanceof IntDocValues);
+    boolean rhsAnInt = (rhs instanceof LongDocValues || rhs instanceof IntDocValues);
+    if (lhsAnInt && rhsAnInt) {
+      return cmp.compare(Long.compare(lhs.longVal(doc), rhs.longVal(doc)));
+    } else {
+      return cmp.compare(Double.compare(lhs.doubleVal(doc), rhs.doubleVal(doc)));
+    }
+  }
+
+  // note: don't override equals; the "name" will be unique and is already compared
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d12b93e2/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java b/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java
index 2a89473..9c51844 100644
--- a/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java
+++ b/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java
@@ -15,6 +15,11 @@
  * limitations under the License.
  */
 package org.apache.solr.search;
+
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.QueryUtils;
 import org.apache.solr.SolrTestCaseJ4;
@@ -24,10 +29,6 @@ import org.apache.solr.response.SolrQueryResponse;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
 
 
 /**
@@ -1075,4 +1076,22 @@ public class QueryEqualityTest extends SolrTestCaseJ4 {
     // assertFuncEquals("agg_multistat(foo_i)", "agg_multistat(foo_i)");
   }
 
+  public void testCompares() throws Exception {
+    assertFuncEquals("gt(foo_i,2)", "gt(foo_i, 2)");
+    assertFuncEquals("gt(foo_i,2)", "gt(foo_i,2)");
+    assertFuncEquals("lt(foo_i,2)", "lt(foo_i,2)");
+    assertFuncEquals("lte(foo_i,2)", "lte(foo_i,2)");
+    assertFuncEquals("gte(foo_i,2)", "gte(foo_i,2)");
+    assertFuncEquals("eq(foo_i,2)", "eq(foo_i,2)");
+
+    boolean equals = false;
+    try {
+      assertFuncEquals("eq(foo_i,2)", "lt(foo_i,2)");
+      equals = true;
+    } catch (AssertionError e) {
+      //expected
+    }
+    assertFalse(equals);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d12b93e2/solr/core/src/test/org/apache/solr/search/function/TestFunctionQuery.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/function/TestFunctionQuery.java b/solr/core/src/test/org/apache/solr/search/function/TestFunctionQuery.java
index f94a9ee..8c65b58 100644
--- a/solr/core/src/test/org/apache/solr/search/function/TestFunctionQuery.java
+++ b/solr/core/src/test/org/apache/solr/search/function/TestFunctionQuery.java
@@ -98,7 +98,7 @@ public class TestFunctionQuery extends SolrTestCaseJ4 {
     return sb.toString();
   }
 
-  void singleTest(String field, String funcTemplate, List<String> args, float... results) {
+  protected void singleTest(String field, String funcTemplate, List<String> args, float... results) {
     String parseableQuery = func(field, funcTemplate);
 
     List<String> nargs = new ArrayList<>(Arrays.asList("q", parseableQuery
@@ -793,4 +793,69 @@ public class TestFunctionQuery extends SolrTestCaseJ4 {
     }
   }
 
-}
+  @Test
+  public void testNumericComparisons() throws Exception {
+    assertU(adoc("id", "1", "age_i", "35"));
+    assertU(adoc("id", "2", "age_i", "25"));
+    assertU(commit());
+
+    // test weighting of functions
+    assertJQ(req("q", "id:1", "fl", "a:gt(age_i,30),b:lt(age_i,30)")
+        , "/response/docs/[0]=={'a':true,'b':false}");
+
+    assertJQ(req("q", "id:1", "fl", "a:exists(gt(foo_i,30))")
+        , "/response/docs/[0]=={'a':false}");
+
+    singleTest("age_i", "if(gt(age_i,30),5,2)",
+               /*id*/1, /*score*/5,
+               /*id*/2, /*score*/2);
+
+    singleTest("age_i", "if(lt(age_i,30),5,2)",
+               /*id*/1, /*score*/2,
+               /*id*/2, /*score*/5);
+
+    singleTest("age_i", "if(lt(age_i,34.5),5,2)",
+               /*id*/1, /*score*/2,
+               /*id*/2, /*score*/5);
+
+    singleTest("age_i", "if(lte(age_i,35),5,2)",
+               /*id*/1, /*score*/5,
+               /*id*/2, /*score*/5);
+
+    singleTest("age_i", "if(gte(age_i,25),5,2)",
+               /*id*/1, /*score*/5,
+               /*id*/2, /*score*/5);
+
+    singleTest("age_i", "if(lte(age_i,25),5,2)",
+               /*id*/1, /*score*/2,
+               /*id*/2, /*score*/5);
+
+    singleTest("age_i", "if(gte(age_i,35),5,2)",
+               /*id*/1, /*score*/5,
+               /*id*/2, /*score*/2);
+
+
+    singleTest("age_i", "if(eq(age_i,30),5,2)",
+               /*id*/1, /*score*/2,
+               /*id*/2, /*score*/2);
+
+    singleTest("age_i", "if(eq(age_i,35),5,2)",
+               /*id*/1, /*score*/5,
+               /*id*/2, /*score*/2);
+  }
+
+  public void testLongComparisons() {
+    assertU(adoc("id", "1", "number_of_atoms_in_universe_l", Long.toString(Long.MAX_VALUE)));
+    assertU(adoc("id", "2", "number_of_atoms_in_universe_l", Long.toString(Long.MAX_VALUE - 1)));
+    assertU(commit());
+
+    singleTest("number_of_atoms_in_universe_l", "if(gt(number_of_atoms_in_universe_l," + Long.toString(Long.MAX_VALUE - 1) + "),5,2)",
+               /*id*/1, /*score*/5,
+               /*id*/2, /*score*/2);
+
+    singleTest("number_of_atoms_in_universe_l", "if(lt(number_of_atoms_in_universe_l," + Long.toString(Long.MAX_VALUE) + "),5,2)",
+               /*id*/2, /*score*/5,
+               /*id*/1, /*score*/2);
+  }
+
+  }


[40/40] lucene-solr:apiv2: Merge branch 'master' into apiv2

Posted by sa...@apache.org.
Merge branch 'master' into apiv2


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

Branch: refs/heads/apiv2
Commit: 1d131592c554e44f7cade97661831759fc0f7c2f
Parents: 003f9b7 d86c369
Author: Steve Rowe <sa...@apache.org>
Authored: Mon Aug 1 21:06:45 2016 -0400
Committer: Steve Rowe <sa...@apache.org>
Committed: Mon Aug 1 21:06:45 2016 -0400

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   18 +
 .../lucene/analysis/minhash/MinHashFilter.java  |    1 +
 .../icu/src/data/uax29/MyanmarSyllable.rbbi     |   50 +
 .../segmentation/DefaultICUTokenizerConfig.java |   14 +-
 .../analysis/icu/segmentation/ICUTokenizer.java |    2 +-
 .../icu/segmentation/ICUTokenizerFactory.java   |    6 +-
 .../icu/segmentation/MyanmarSyllable.brk        |  Bin 0 -> 19776 bytes
 .../icu/segmentation/TestICUTokenizer.java      |    6 +-
 .../icu/segmentation/TestICUTokenizerCJK.java   |    2 +-
 .../icu/segmentation/TestMyanmarSyllable.java   |  156 ++
 .../segmentation/TestWithCJKBigramFilter.java   |    4 +-
 .../classification/utils/DatasetSplitter.java   |   14 +-
 .../lucene/codecs/MutablePointsReader.java      |   39 +
 .../codecs/lucene60/Lucene60PointsWriter.java   |    9 +
 .../org/apache/lucene/document/FieldType.java   |    2 +-
 .../apache/lucene/index/DocumentsWriter.java    |   11 -
 .../index/DocumentsWriterFlushControl.java      |   27 +-
 .../index/DocumentsWriterStallControl.java      |   26 +-
 .../apache/lucene/index/PointValuesWriter.java  |  160 +-
 .../java/org/apache/lucene/util/ArrayUtil.java  |   71 +-
 .../org/apache/lucene/util/ByteBlockPool.java   |    8 +
 .../org/apache/lucene/util/IntroSelector.java   |  126 ++
 .../org/apache/lucene/util/IntroSorter.java     |    7 +-
 .../org/apache/lucene/util/RadixSelector.java   |  202 +++
 .../java/org/apache/lucene/util/Selector.java   |   41 +
 .../org/apache/lucene/util/bkd/BKDWriter.java   |  489 ++++--
 .../util/bkd/MutablePointsReaderUtils.java      |  185 +++
 .../index/TestDocumentsWriterStallControl.java  |    6 +-
 .../apache/lucene/index/TestPointValues.java    |    3 +-
 .../apache/lucene/util/TestByteBlockPool.java   |    8 +-
 .../apache/lucene/util/TestIntroSelector.java   |   86 ++
 .../apache/lucene/util/TestRadixSelector.java   |   77 +
 .../util/bkd/TestMutablePointsReaderUtils.java  |  251 +++
 lucene/ivy-versions.properties                  |    1 +
 .../apache/lucene/index/memory/MemoryIndex.java |   24 +-
 .../lucene/index/memory/TestMemoryIndex.java    |   14 +
 .../valuesource/ComparisonBoolFunction.java     |  105 ++
 .../org/apache/lucene/replicator/nrt/Node.java  |   19 +-
 .../lucene/replicator/nrt/PrimaryNode.java      |    5 +
 .../lucene/document/DoubleRangeField.java       |  262 ++++
 .../apache/lucene/document/RangeFieldQuery.java |  315 ++++
 .../lucene/document/TestDoubleRangeField.java   |   67 +
 .../search/BaseRangeFieldQueryTestCase.java     |  427 +++++
 .../search/TestDoubleRangeFieldQueries.java     |  106 ++
 .../codecs/asserting/AssertingPointsFormat.java |    1 +
 solr/CHANGES.txt                                |   29 +
 .../apache/solr/hadoop/ZooKeeperInspector.java  |   21 +-
 .../solr/hadoop/MorphlineBasicMiniMRTest.java   |    2 +
 .../solr/hadoop/MorphlineGoLiveMiniMRTest.java  |    2 +
 solr/core/ivy.xml                               |    4 +
 .../solr/core/IndexDeletionPolicyWrapper.java   |   27 +-
 .../src/java/org/apache/solr/core/SolrCore.java |   36 +-
 .../core/snapshots/SolrSnapshotManager.java     |  134 ++
 .../snapshots/SolrSnapshotMetaDataManager.java  |  416 +++++
 .../solr/core/snapshots/package-info.java       |   22 +
 .../org/apache/solr/handler/IndexFetcher.java   |   16 +-
 .../apache/solr/handler/ReplicationHandler.java |   24 +-
 .../apache/solr/handler/RequestHandlerBase.java |   20 +-
 .../org/apache/solr/handler/RestoreCore.java    |   18 +-
 .../org/apache/solr/handler/SQLHandler.java     |    5 +-
 .../org/apache/solr/handler/SnapShooter.java    |   39 +-
 .../solr/handler/admin/CoreAdminHandler.java    |    9 +-
 .../solr/handler/admin/CoreAdminHandlerApi.java |    4 +-
 .../solr/handler/admin/CoreAdminOperation.java  | 1459 +++++++++---------
 .../java/org/apache/solr/search/LRUCache.java   |   34 +-
 .../apache/solr/search/ValueSourceParser.java   |   52 +
 .../function/SolrComparisonBoolFunction.java    |   58 +
 .../solr/security/AuthenticationPlugin.java     |   29 +-
 .../apache/solr/security/BasicAuthPlugin.java   |    5 +-
 .../security/DelegationTokenKerberosFilter.java |  171 ++
 .../apache/solr/security/KerberosFilter.java    |   14 +
 .../apache/solr/security/KerberosPlugin.java    |  195 ++-
 .../solr/security/PKIAuthenticationPlugin.java  |   13 +-
 .../solr/security/PrintWriterWrapper.java       |  215 +++
 .../apache/solr/servlet/SolrDispatchFilter.java |   12 +-
 .../solr/update/DirectUpdateHandler2.java       |  115 +-
 .../apache/solr/update/IndexFingerprint.java    |    2 +
 .../apache/solr/util/ConcurrentLRUCache.java    |   33 +-
 .../solr/cloud/ChaosMonkeyShardSplitTest.java   |   13 +-
 .../apache/solr/cloud/KerberosTestServices.java |  228 +++
 .../org/apache/solr/cloud/KerberosTestUtil.java |  147 --
 .../apache/solr/cloud/LeaderElectionTest.java   |    1 +
 ...utOfBoxZkACLAndCredentialsProvidersTest.java |    7 +-
 ...rriddenZkACLAndCredentialsProvidersTest.java |   71 +-
 .../org/apache/solr/cloud/OverseerTest.java     |    1 +
 .../solr/cloud/SaslZkACLProviderTest.java       |   39 +-
 .../solr/cloud/TestAuthenticationFramework.java |   10 +-
 .../cloud/TestLocalFSCloudBackupRestore.java    |    2 +
 .../cloud/TestMiniSolrCloudClusterKerberos.java |   29 +-
 .../TestSolrCloudWithDelegationTokens.java      |  405 +++++
 .../cloud/TestSolrCloudWithKerberosAlt.java     |   37 +-
 ...MParamsZkACLAndCredentialsProvidersTest.java |   25 +-
 .../solr/cloud/overseer/ZkStateWriterTest.java  |  393 ++---
 .../core/snapshots/TestSolrCoreSnapshots.java   |  419 +++++
 .../apache/solr/handler/BackupRestoreUtils.java |   37 +
 .../solr/handler/TestHdfsBackupRestoreCore.java |   46 +-
 .../solr/handler/TestReplicationHandler.java    |   20 +-
 .../apache/solr/search/QueryEqualityTest.java   |   27 +-
 .../solr/search/function/TestFunctionQuery.java |   69 +-
 ...ramDelegationTokenAuthenticationHandler.java |  109 ++
 .../solr/security/MockAuthenticationPlugin.java |   32 +-
 .../solr/update/DirectUpdateHandlerTest.java    |   48 +-
 solr/licenses/curator-recipes-2.8.0.jar.sha1    |    1 +
 solr/licenses/curator-recipes-LICENSE-ASL.txt   |  202 +++
 solr/licenses/curator-recipes-NOTICE.txt        |    5 +
 solr/solrj/ivy.xml                              |    5 +
 .../solr/client/solrj/impl/CloudSolrClient.java |   25 +-
 .../solr/client/solrj/impl/HttpSolrClient.java  |   57 +-
 .../solrj/impl/Krb5HttpClientBuilder.java       |   18 +-
 .../client/solrj/io/stream/CloudSolrStream.java |   25 +-
 .../client/solrj/io/stream/TopicStream.java     |   16 +-
 .../client/solrj/request/CoreAdminRequest.java  |   57 +
 .../solrj/request/DelegationTokenRequest.java   |  152 ++
 .../solrj/response/DelegationTokenResponse.java |  108 ++
 .../apache/solr/common/cloud/ClusterState.java  |   13 +-
 .../solr/common/cloud/ConnectionManager.java    |   15 +-
 .../common/cloud/DefaultConnectionStrategy.java |    4 +-
 .../solr/common/cloud/SaslZkACLProvider.java    |   21 +-
 .../cloud/SecurityAwareZkACLProvider.java       |   79 +
 .../apache/solr/common/cloud/SolrZkClient.java  |    2 +-
 ...ParamsAllAndReadonlyDigestZkACLProvider.java |   52 +-
 .../cloud/ZkClientConnectionStrategy.java       |    4 +-
 .../apache/solr/common/cloud/ZkStateReader.java |   21 +
 .../solr/common/params/CoreAdminParams.java     |   10 +-
 .../request/TestDelegationTokenRequest.java     |   70 +
 .../response/TestDelegationTokenResponse.java   |  138 ++
 solr/webapp/web/css/angular/cloud.css           |   14 +
 solr/webapp/web/js/angular/app.js               |    2 +-
 solr/webapp/web/js/angular/controllers/cloud.js |    5 +
 solr/webapp/web/js/angular/controllers/query.js |   10 +-
 solr/webapp/web/partials/cloud.html             |    3 +-
 131 files changed, 8224 insertions(+), 1813 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1d131592/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1d131592/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
----------------------------------------------------------------------
diff --cc solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
index d2742bc,d28957c..00b18fa
--- a/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
+++ b/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
@@@ -18,10 -18,8 +18,10 @@@ package org.apache.solr.handler
  
  import java.lang.invoke.MethodHandles;
  import java.net.URL;
 +import java.util.Collection;
- import java.util.concurrent.atomic.AtomicLong;
+ import java.util.concurrent.atomic.LongAdder;
  
 +import com.google.common.collect.ImmutableList;
  import org.apache.solr.common.SolrException;
  import org.apache.solr.common.params.SolrParams;
  import org.apache.solr.common.util.NamedList;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1d131592/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1d131592/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandlerApi.java
----------------------------------------------------------------------
diff --cc solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandlerApi.java
index 8ff170b,0000000..9173ad2
mode 100644,000000..100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandlerApi.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandlerApi.java
@@@ -1,159 -1,0 +1,159 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one or more
 + * contributor license agreements.  See the NOTICE file distributed with
 + * this work for additional information regarding copyright ownership.
 + * The ASF licenses this file to You under the Apache License, Version 2.0
 + * (the "License"); you may not use this file except in compliance with
 + * the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +
 +package org.apache.solr.handler.admin;
 +
 +import java.util.Arrays;
 +import java.util.Collections;
 +import java.util.List;
 +import java.util.Locale;
 +import java.util.Map;
 +
 +import com.google.common.collect.ImmutableMap;
 +import org.apache.solr.client.solrj.SolrRequest;
 +import org.apache.solr.request.SolrQueryRequest;
 +import org.apache.solr.response.SolrQueryResponse;
 +import org.apache.solr.util.CommandOperation;
 +
 +import static org.apache.solr.client.solrj.SolrRequest.METHOD.*;
 +import static org.apache.solr.handler.admin.CoreAdminHandlerApi.EndPoint.NODEAPIS;
 +import static org.apache.solr.handler.admin.CoreAdminHandlerApi.EndPoint.NODEINVOKE;
 +import static org.apache.solr.handler.admin.CoreAdminOperation.*;
 +import static org.apache.solr.handler.admin.CoreAdminHandlerApi.EndPoint.CORES_COMMANDS;
 +import static org.apache.solr.handler.admin.CoreAdminHandlerApi.EndPoint.CORES_STATUS;
 +import static org.apache.solr.handler.admin.CoreAdminHandlerApi.EndPoint.PER_CORE_COMMANDS;
 +
 +public class CoreAdminHandlerApi extends BaseHandlerApiSupport {
 +  private final CoreAdminHandler handler;
 +
 +  public CoreAdminHandlerApi(CoreAdminHandler handler) {
 +    this.handler = handler;
 +  }
 +
 +  enum Cmd implements ApiCommand<CoreAdminHandlerApi> {
 +    CREATE(CORES_COMMANDS, POST, CREATE_OP, null, ImmutableMap.of("config", "configSet")),
 +    UNLOAD(PER_CORE_COMMANDS, POST, UNLOAD_OP, null, null),
 +    RELOAD(PER_CORE_COMMANDS, POST, RELOAD_OP, null, null),
 +    STATUS(CORES_STATUS, GET, STATUS_OP),
 +    SWAP(PER_CORE_COMMANDS, POST, SWAP_OP, null, ImmutableMap.of("other", "with")),
 +    RENAME(PER_CORE_COMMANDS, POST, RENAME_OP, null, null),
 +    MERGEINDEXES(PER_CORE_COMMANDS, POST, MERGEINDEXES_OP, null, null),
 +    SPLIT(PER_CORE_COMMANDS, POST, SPLIT_OP, null, ImmutableMap.of("split.key", "splitKey")),
 +    PREPRECOVERY(PER_CORE_COMMANDS, POST, PREPRECOVERY_OP, null, null),
 +    REQUESTRECOVERY(PER_CORE_COMMANDS, POST, REQUESTRECOVERY_OP, null, null),
 +    REQUESTSYNCSHARD(PER_CORE_COMMANDS, POST, REQUESTRECOVERY_OP, null, null),
 +    REQUESTBUFFERUPDATES(PER_CORE_COMMANDS, POST, REQUESTBUFFERUPDATES_OP, null, null),
 +    REQUESTAPPLYUPDATES(PER_CORE_COMMANDS, POST, REQUESTAPPLYUPDATES_OP, null, null),
 +    REQUESTSTATUS(PER_CORE_COMMANDS, POST, REQUESTSTATUS_OP, null, null),
 +    OVERSEEROP(NODEAPIS, POST, OVERSEEROP_OP, null, null),
 +    REJOINLEADERELECTION(NODEAPIS, POST, REJOINLEADERELECTION_OP, null, null),
 +    INVOKE(NODEINVOKE, GET, INVOKE_OP, null, null),
 +    FORCEPREPAREFORLEADERSHIP(PER_CORE_COMMANDS, POST, FORCEPREPAREFORLEADERSHIP_OP, null, null);
 +
 +    public final String commandName;
 +    public final EndPoint endPoint;
 +    public final SolrRequest.METHOD method;
 +    public final Map<String, String> paramstoAttr;
 +    final CoreAdminOperation target;
 +
 +
 +    Cmd(EndPoint endPoint, SolrRequest.METHOD method, CoreAdminOperation target) {
 +      this.endPoint = endPoint;
 +      this.method = method;
 +      this.target = target;
 +      commandName = null;
 +      paramstoAttr = Collections.EMPTY_MAP;
 +
 +    }
 +
 +
 +    Cmd(EndPoint endPoint, SolrRequest.METHOD method, CoreAdminOperation target, String commandName,
 +        Map<String, String> paramstoAttr) {
 +      this.commandName = commandName == null ? target.action.toString().toLowerCase(Locale.ROOT) : commandName;
 +      this.endPoint = endPoint;
 +      this.method = method;
 +      this.target = target;
 +      this.paramstoAttr = paramstoAttr == null ? Collections.EMPTY_MAP : paramstoAttr;
 +    }
 +
 +    @Override
 +    public String getName() {
 +      return commandName;
 +    }
 +
 +    @Override
 +    public SolrRequest.METHOD getHttpMethod() {
 +      return method;
 +    }
 +
 +    @Override
 +    public V2EndPoint getEndPoint() {
 +      return endPoint;
 +    }
 +
 +    @Override
 +    public String getParamSubstitute(String param) {
 +      return paramstoAttr.containsKey(param) ? paramstoAttr.get(param) : param;
 +    }
 +  }
 +
 +
 +
 +  enum EndPoint implements V2EndPoint {
 +    CORES_STATUS("cores.Status"),
 +    CORES_COMMANDS("cores.Commands"),
 +    PER_CORE_COMMANDS("cores.core.Commands"),
 +    NODEINVOKE("node.invoke"),
 +    NODEAPIS("node.Commands")
 +    ;
 +
 +    final String specName;
 +
 +    EndPoint(String specName) {
 +      this.specName = specName;
 +    }
 +
 +    @Override
 +    public String getSpecName() {
 +      return specName;
 +    }
 +  }
 +
 +
 +  @Override
 +  protected void invokeCommand(SolrQueryRequest req, SolrQueryResponse rsp, ApiCommand command,
 +                               CommandOperation c) throws Exception {
-     ((Cmd) command).target.call(new CoreAdminHandler.CallInfo(handler, req, rsp, ((Cmd) command).target));
++    ((Cmd) command).target.execute(new CoreAdminHandler.CallInfo(handler, req, rsp, ((Cmd) command).target));
 +  }
 +
 +  @Override
 +  protected void invokeUrl(ApiCommand command, SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
-     ((Cmd) command).target.call(new CoreAdminHandler.CallInfo(handler, req, rsp, ((Cmd) command).target));
++    ((Cmd) command).target.execute(new CoreAdminHandler.CallInfo(handler, req, rsp, ((Cmd) command).target));
 +  }
 +
 +  @Override
 +  protected List<ApiCommand> getCommands() {
 +    return Arrays.asList(Cmd.values());
 +  }
 +
 +  @Override
 +  protected List<V2EndPoint> getEndPoints() {
 +    return Arrays.asList(EndPoint.values());
 +  }
 +
 +
 +}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1d131592/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1d131592/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1d131592/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1d131592/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
----------------------------------------------------------------------


[06/40] lucene-solr:apiv2: LUCENE-7381: Fix equals relation in RangeFieldQuery. Fix relation logic in BaseRangeFieldQueryTestCase.

Posted by sa...@apache.org.
LUCENE-7381: Fix equals relation in RangeFieldQuery. Fix relation logic in BaseRangeFieldQueryTestCase.


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

Branch: refs/heads/apiv2
Commit: 7f1db8a047818da337b27fe9dce0824cb5a02b96
Parents: f70adac
Author: Nicholas Knize <nk...@gmail.com>
Authored: Fri Jul 22 10:59:48 2016 -0500
Committer: Nicholas Knize <nk...@gmail.com>
Committed: Fri Jul 22 11:21:17 2016 -0500

----------------------------------------------------------------------
 .../apache/lucene/document/RangeFieldQuery.java | 16 ++++++-----
 .../search/BaseRangeFieldQueryTestCase.java     | 28 ++++++++++++++++++--
 2 files changed, 35 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7f1db8a0/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java b/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java
index 36de9b2..1fa894f 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java
@@ -124,8 +124,9 @@ abstract class RangeFieldQuery extends Query {
               @Override
               public void visit(int docID, byte[] leaf) throws IOException {
                 // add the document iff:
-                if (// target is within cell and queryType is INTERSECTS or CONTAINS:
-                    (comparator.isWithin(leaf) && queryType != QueryType.WITHIN)
+                if (Arrays.equals(ranges, leaf)
+                    // target is within cell and queryType is INTERSECTS or CONTAINS:
+                    || (comparator.isWithin(leaf) && queryType != QueryType.WITHIN)
                     // target contains cell and queryType is INTERSECTS or WITHIN:
                     || (comparator.contains(leaf) && queryType != QueryType.CONTAINS)
                     // target is not disjoint (crosses) and queryType is INTERSECTS
@@ -139,12 +140,12 @@ abstract class RangeFieldQuery extends Query {
                 // compute range relation for BKD traversal
                 if (comparator.isDisjoint(node)) {
                   return Relation.CELL_OUTSIDE_QUERY;
-                } else if (comparator.contains(node)) {
-                  // target contains cell; add iff queryType is not a CONTAINS query:
-                  return (queryType == QueryType.CONTAINS) ? Relation.CELL_OUTSIDE_QUERY : Relation.CELL_INSIDE_QUERY;
                 } else if (comparator.isWithin(node)) {
                   // target within cell; continue traversing:
                   return Relation.CELL_CROSSES_QUERY;
+                } else if (comparator.contains(node)) {
+                  // target contains cell; add iff queryType is not a CONTAINS query:
+                  return (queryType == QueryType.CONTAINS) ? Relation.CELL_OUTSIDE_QUERY : Relation.CELL_INSIDE_QUERY;
                 }
                 // target intersects cell; continue traversing:
                 return Relation.CELL_CROSSES_QUERY;
@@ -170,8 +171,9 @@ abstract class RangeFieldQuery extends Query {
         if (values.getDocCount(field) == reader.maxDoc()) {
           // if query crosses, docs need to be further scrutinized
           byte[] range = getInternalRange(values.getMinPackedValue(field), values.getMaxPackedValue(field));
-          // if the internal node is not contained by the query, all docs do not match
-          if (((comparator.contains(range) && queryType == QueryType.CONTAINS)) == false) {
+          // if the internal node is not equal and not contained by the query, all docs do not match
+          if ((!Arrays.equals(ranges, range)
+              && (comparator.contains(range) && queryType != QueryType.CONTAINS)) == false) {
             allDocsMatch = false;
           }
         } else {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7f1db8a0/lucene/sandbox/src/test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java b/lucene/sandbox/src/test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java
index dadb588..d9cb830 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java
@@ -17,6 +17,7 @@
 package org.apache.lucene.search;
 
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.HashSet;
 import java.util.Set;
 
@@ -262,10 +263,11 @@ public abstract class BaseRangeFieldQueryTestCase extends LuceneTestCase {
 
         if (hits.get(docID) != expected) {
           StringBuilder b = new StringBuilder();
+          b.append("FAIL (iter " + iter + "): ");
           if (expected == true) {
-            b.append("FAILS: id=" + id + (boxes[id].length > 1 ? " (MultiValue) " : " ") + "should match but did not\n");
+            b.append("id=" + id + (boxes[id].length > 1 ? " (MultiValue) " : " ") + "should match but did not\n");
           } else {
-            b.append("FAIL: id=" + id + " should not match but did\n");
+            b.append("id=" + id + " should not match but did\n");
           }
           b.append(" queryBox=" + queryBox + "\n");
           b.append(" box" + ((boxes[id].length > 1) ? "es=" : "=" ) + boxes[id][0]);
@@ -292,6 +294,9 @@ public abstract class BaseRangeFieldQueryTestCase extends LuceneTestCase {
   }
 
   protected boolean expectedBBoxQueryResult(Box queryBox, Box box, Box.QueryType queryType) {
+    if (box.equals(queryBox)) {
+      return true;
+    }
     Box.QueryType relation = box.relate(queryBox);
     if (queryType == Box.QueryType.INTERSECTS) {
       return relation != null;
@@ -345,6 +350,25 @@ public abstract class BaseRangeFieldQueryTestCase extends LuceneTestCase {
       }
     }
 
+    @Override
+    public boolean equals(Object o) {
+      return o != null
+          && getClass() == o.getClass()
+          && equalTo(getClass().cast(o));
+    }
+
+    private boolean equalTo(Box o) {
+      return Arrays.equals(min, o.min)
+          && Arrays.equals(max, o.max);
+    }
+
+    @Override
+    public int hashCode() {
+      int result = Arrays.hashCode(min);
+      result = 31 * result + Arrays.hashCode(max);
+      return result;
+    }
+
     QueryType relate(Box other) {
       // check disjoint
       for (int d=0; d<this.min.length; ++d) {


[19/40] lucene-solr:apiv2: SOLR-9335: Switch Solr cache/search/update stats counters to use LongAdder

Posted by sa...@apache.org.
SOLR-9335: Switch Solr cache/search/update stats counters to use LongAdder


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

Branch: refs/heads/apiv2
Commit: f906862afe2ba4ef7ac521555ee59a86ef0b2a75
Parents: aba731a
Author: Varun Thacker <va...@apache.org>
Authored: Tue Jul 26 16:17:32 2016 +0530
Committer: Varun Thacker <va...@apache.org>
Committed: Tue Jul 26 16:17:41 2016 +0530

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../apache/solr/handler/RequestHandlerBase.java |  26 ++---
 .../java/org/apache/solr/search/LRUCache.java   |  34 +++---
 .../solr/update/DirectUpdateHandler2.java       | 115 +++++++++----------
 .../apache/solr/util/ConcurrentLRUCache.java    |  33 +++---
 .../solr/update/DirectUpdateHandlerTest.java    |  48 ++++----
 6 files changed, 130 insertions(+), 129 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f906862a/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 4da2d0e..64c4291 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -174,6 +174,9 @@ Optimizations
 * SOLR-9264: Optimize ZkController.publishAndWaitForDownStates to not read all collection states and
   watch relevant collections instead. (Hrishikesh Gadre, shalin)
 
+* SOLR-9335: Solr cache/search/update stats counters now use LongAdder which are supposed to have higher throughput
+  under high contention. (Varun Thacker)
+
 Other Changes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f906862a/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java b/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
index bb7df32..d28957c 100644
--- a/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
+++ b/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
@@ -16,13 +16,17 @@
  */
 package org.apache.solr.handler;
 
+import java.lang.invoke.MethodHandles;
+import java.net.URL;
+import java.util.concurrent.atomic.LongAdder;
+
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.SuppressForbidden;
-import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.PluginBag;
+import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.SolrInfoMBean;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestHandler;
@@ -35,10 +39,6 @@ import org.apache.solr.util.stats.TimerContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.lang.invoke.MethodHandles;
-import java.net.URL;
-import java.util.concurrent.atomic.AtomicLong;
-
 import static org.apache.solr.core.RequestParams.USEPARAM;
 
 /**
@@ -53,10 +53,10 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
   protected boolean httpCaching = true;
 
   // Statistics
-  private final AtomicLong numRequests = new AtomicLong();
-  private final AtomicLong numServerErrors = new AtomicLong();
-  private final AtomicLong numClientErrors = new AtomicLong();
-  private final AtomicLong numTimeouts = new AtomicLong();
+  private final LongAdder numRequests = new LongAdder();
+  private final LongAdder numServerErrors = new LongAdder();
+  private final LongAdder numClientErrors = new LongAdder();
+  private final LongAdder numTimeouts = new LongAdder();
   private final Timer requestTimes = new Timer();
 
   private final long handlerStart;
@@ -144,7 +144,7 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
 
   @Override
   public void handleRequest(SolrQueryRequest req, SolrQueryResponse rsp) {
-    numRequests.incrementAndGet();
+    numRequests.increment();
     TimerContext timer = requestTimes.time();
     try {
       if(pluginInfo != null && pluginInfo.attributes.containsKey(USEPARAM)) req.getContext().put(USEPARAM,pluginInfo.attributes.get(USEPARAM));
@@ -158,7 +158,7 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
         Object partialResults = header.get(SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_KEY);
         boolean timedOut = partialResults == null ? false : (Boolean)partialResults;
         if( timedOut ) {
-          numTimeouts.incrementAndGet();
+          numTimeouts.increment();
           rsp.setHttpCaching(false);
         }
       }
@@ -185,9 +185,9 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
         SolrException.log(log, e);
 
         if (isServerError) {
-          numServerErrors.incrementAndGet();
+          numServerErrors.increment();
         } else {
-          numClientErrors.incrementAndGet();
+          numClientErrors.increment();
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f906862a/solr/core/src/java/org/apache/solr/search/LRUCache.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/LRUCache.java b/solr/core/src/java/org/apache/solr/search/LRUCache.java
index 8e030fd..0d9f406 100644
--- a/solr/core/src/java/org/apache/solr/search/LRUCache.java
+++ b/solr/core/src/java/org/apache/solr/search/LRUCache.java
@@ -23,7 +23,7 @@ import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.LongAdder;
 
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.Accountables;
@@ -61,11 +61,11 @@ public class LRUCache<K,V> extends SolrCacheBase implements SolrCache<K,V>, Acco
    * of an LRUCache at the same time.  Make sure everything is thread safe.
    */
   private static class CumulativeStats {
-    AtomicLong lookups = new AtomicLong();
-    AtomicLong hits = new AtomicLong();
-    AtomicLong inserts = new AtomicLong();
-    AtomicLong evictions = new AtomicLong();
-    AtomicLong evictionsRamUsage = new AtomicLong();
+    LongAdder lookups = new LongAdder();
+    LongAdder hits = new LongAdder();
+    LongAdder inserts = new LongAdder();
+    LongAdder evictions = new LongAdder();
+    LongAdder evictionsRamUsage = new LongAdder();
   }
 
   private CumulativeStats stats;
@@ -124,8 +124,8 @@ public class LRUCache<K,V> extends SolrCacheBase implements SolrCache<K,V>, Acco
                 iterator.remove();
                 evictions++;
                 evictionsRamUsage++;
-                stats.evictions.incrementAndGet();
-                stats.evictionsRamUsage.incrementAndGet();
+                stats.evictions.increment();
+                stats.evictionsRamUsage.increment();
               } while (iterator.hasNext() && ramBytesUsed > maxRamBytes);
               // must return false according to javadocs of removeEldestEntry if we're modifying
               // the map ourselves
@@ -135,7 +135,7 @@ public class LRUCache<K,V> extends SolrCacheBase implements SolrCache<K,V>, Acco
               // this doesn't need to be synchronized because it will
               // only be called in the context of a higher level synchronized block.
               evictions++;
-              stats.evictions.incrementAndGet();
+              stats.evictions.increment();
               return true;
             }
           }
@@ -180,7 +180,7 @@ public class LRUCache<K,V> extends SolrCacheBase implements SolrCache<K,V>, Acco
   public V put(K key, V value) {
     synchronized (map) {
       if (getState() == State.LIVE) {
-        stats.inserts.incrementAndGet();
+        stats.inserts.increment();
       }
 
       // increment local inserts regardless of state???
@@ -232,10 +232,10 @@ public class LRUCache<K,V> extends SolrCacheBase implements SolrCache<K,V>, Acco
       if (getState() == State.LIVE) {
         // only increment lookups and hits if we are live.
         lookups++;
-        stats.lookups.incrementAndGet();
+        stats.lookups.increment();
         if (val!=null) {
           hits++;
-          stats.hits.incrementAndGet();
+          stats.hits.increment();
         }
       }
       return val;
@@ -341,15 +341,15 @@ public class LRUCache<K,V> extends SolrCacheBase implements SolrCache<K,V>, Acco
     }
     lst.add("warmupTime", warmupTime);
     
-    long clookups = stats.lookups.get();
-    long chits = stats.hits.get();
+    long clookups = stats.lookups.longValue();
+    long chits = stats.hits.longValue();
     lst.add("cumulative_lookups", clookups);
     lst.add("cumulative_hits", chits);
     lst.add("cumulative_hitratio", calcHitRatio(clookups, chits));
-    lst.add("cumulative_inserts", stats.inserts.get());
-    lst.add("cumulative_evictions", stats.evictions.get());
+    lst.add("cumulative_inserts", stats.inserts.longValue());
+    lst.add("cumulative_evictions", stats.evictions.longValue());
     if (maxRamBytes != Long.MAX_VALUE)  {
-      lst.add("cumulative_evictionsRamUsage", stats.evictionsRamUsage.get());
+      lst.add("cumulative_evictionsRamUsage", stats.evictionsRamUsage.longValue());
     }
     
     return lst;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f906862a/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java b/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
index 8c3c749..f0794a6 100644
--- a/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
+++ b/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
@@ -26,7 +26,7 @@ import java.util.Locale;
 import java.util.Map;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
-import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.LongAdder;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.CodecReader;
@@ -76,20 +76,20 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
   protected final SolrCoreState solrCoreState;
 
   // stats
-  AtomicLong addCommands = new AtomicLong();
-  AtomicLong addCommandsCumulative = new AtomicLong();
-  AtomicLong deleteByIdCommands= new AtomicLong();
-  AtomicLong deleteByIdCommandsCumulative= new AtomicLong();
-  AtomicLong deleteByQueryCommands= new AtomicLong();
-  AtomicLong deleteByQueryCommandsCumulative= new AtomicLong();
-  AtomicLong expungeDeleteCommands = new AtomicLong();
-  AtomicLong mergeIndexesCommands = new AtomicLong();
-  AtomicLong commitCommands= new AtomicLong();
-  AtomicLong optimizeCommands= new AtomicLong();
-  AtomicLong rollbackCommands= new AtomicLong();
-  AtomicLong numDocsPending= new AtomicLong();
-  AtomicLong numErrors = new AtomicLong();
-  AtomicLong numErrorsCumulative = new AtomicLong();
+  LongAdder addCommands = new LongAdder();
+  LongAdder addCommandsCumulative = new LongAdder();
+  LongAdder deleteByIdCommands= new LongAdder();
+  LongAdder deleteByIdCommandsCumulative= new LongAdder();
+  LongAdder deleteByQueryCommands= new LongAdder();
+  LongAdder deleteByQueryCommandsCumulative= new LongAdder();
+  LongAdder expungeDeleteCommands = new LongAdder();
+  LongAdder mergeIndexesCommands = new LongAdder();
+  LongAdder commitCommands= new LongAdder();
+  LongAdder optimizeCommands= new LongAdder();
+  LongAdder rollbackCommands= new LongAdder();
+  LongAdder numDocsPending= new LongAdder();
+  LongAdder numErrors = new LongAdder();
+  LongAdder numErrorsCumulative = new LongAdder();
 
   // tracks when auto-commit should occur
   protected final CommitTracker commitTracker;
@@ -158,7 +158,7 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
   }
 
   protected void rollbackWriter() throws IOException {
-    numDocsPending.set(0);
+    numDocsPending.reset();
     solrCoreState.rollbackIndexWriter(core);
     
   }
@@ -192,8 +192,8 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
   private int addDoc0(AddUpdateCommand cmd) throws IOException {
     int rc = -1;
 
-    addCommands.incrementAndGet();
-    addCommandsCumulative.incrementAndGet();
+    addCommands.increment();
+    addCommandsCumulative.increment();
 
     // if there is no ID field, don't overwrite
     if (idField == null) {
@@ -230,10 +230,10 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
       rc = 1;
     } finally {
       if (rc != 1) {
-        numErrors.incrementAndGet();
-        numErrorsCumulative.incrementAndGet();
+        numErrors.increment();
+        numErrorsCumulative.increment();
       } else {
-        numDocsPending.incrementAndGet();
+        numDocsPending.increment();
       }
     }
 
@@ -368,8 +368,8 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
   // we don't return the number of docs deleted because it's not always possible to quickly know that info.
   @Override
   public void delete(DeleteUpdateCommand cmd) throws IOException {
-    deleteByIdCommands.incrementAndGet();
-    deleteByIdCommandsCumulative.incrementAndGet();
+    deleteByIdCommands.increment();
+    deleteByIdCommandsCumulative.increment();
 
     Term deleteTerm = new Term(idField.getName(), cmd.getIndexedId());
     // SolrCore.verbose("deleteDocuments",deleteTerm,writer);
@@ -426,8 +426,8 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
   // we don't return the number of docs deleted because it's not always possible to quickly know that info.
   @Override
   public void deleteByQuery(DeleteUpdateCommand cmd) throws IOException {
-    deleteByQueryCommands.incrementAndGet();
-    deleteByQueryCommandsCumulative.incrementAndGet();
+    deleteByQueryCommands.increment();
+    deleteByQueryCommandsCumulative.increment();
     boolean madeIt=false;
     try {
       Query q = getQuery(cmd);
@@ -473,8 +473,8 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
 
     } finally {
       if (!madeIt) {
-        numErrors.incrementAndGet();
-        numErrorsCumulative.incrementAndGet();
+        numErrors.increment();
+        numErrorsCumulative.increment();
       }
     }
   }
@@ -482,7 +482,7 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
 
   @Override
   public int mergeIndexes(MergeIndexesCommand cmd) throws IOException {
-    mergeIndexesCommands.incrementAndGet();
+    mergeIndexesCommands.increment();
     int rc;
 
     log.info("start " + cmd);
@@ -545,7 +545,7 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
       error=false;
     }
     finally {
-      if (error) numErrors.incrementAndGet();
+      if (error) numErrors.increment();
     }
   }
 
@@ -557,10 +557,10 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
     }
 
     if (cmd.optimize) {
-      optimizeCommands.incrementAndGet();
+      optimizeCommands.increment();
     } else {
-      commitCommands.incrementAndGet();
-      if (cmd.expungeDeletes) expungeDeleteCommands.incrementAndGet();
+      commitCommands.increment();
+      if (cmd.expungeDeletes) expungeDeleteCommands.increment();
     }
 
     Future[] waitSearcher = null;
@@ -622,7 +622,7 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
           }
 
           // SolrCore.verbose("writer.commit() end");
-          numDocsPending.set(0);
+          numDocsPending.reset();
           callPostCommitCallbacks();
         }
       } finally {
@@ -676,10 +676,10 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
         solrCoreState.getCommitLock().unlock();
       }
 
-      addCommands.set(0);
-      deleteByIdCommands.set(0);
-      deleteByQueryCommands.set(0);
-      if (error) numErrors.incrementAndGet();
+      addCommands.reset();
+      deleteByIdCommands.reset();
+      deleteByQueryCommands.reset();
+      if (error) numErrors.increment();
     }
 
     // if we are supposed to wait for the searcher to be registered, then we should do it
@@ -707,7 +707,7 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
       throw new UnsupportedOperationException("Rollback is currently not supported in SolrCloud mode. (SOLR-4895)");
     }
 
-    rollbackCommands.incrementAndGet();
+    rollbackCommands.increment();
 
     boolean error=true;
 
@@ -727,13 +727,10 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
       error=false;
     }
     finally {
-      addCommandsCumulative.set(
-          addCommandsCumulative.get() - addCommands.getAndSet( 0 ) );
-      deleteByIdCommandsCumulative.set(
-          deleteByIdCommandsCumulative.get() - deleteByIdCommands.getAndSet( 0 ) );
-      deleteByQueryCommandsCumulative.set(
-          deleteByQueryCommandsCumulative.get() - deleteByQueryCommands.getAndSet( 0 ) );
-      if (error) numErrors.incrementAndGet();
+      addCommandsCumulative.add(-addCommands.sumThenReset());
+      deleteByIdCommandsCumulative.add(-deleteByIdCommands.sumThenReset());
+      deleteByQueryCommandsCumulative.add(-deleteByQueryCommands.sumThenReset());
+      if (error) numErrors.increment();
     }
   }
 
@@ -749,7 +746,7 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
     commitTracker.close();
     softCommitTracker.close();
 
-    numDocsPending.set(0);
+    numDocsPending.reset();
   }
 
 
@@ -882,7 +879,7 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
   @Override
   public NamedList getStatistics() {
     NamedList lst = new SimpleOrderedMap();
-    lst.add("commits", commitCommands.get());
+    lst.add("commits", commitCommands.longValue());
     if (commitTracker.getDocsUpperBound() > 0) {
       lst.add("autocommit maxDocs", commitTracker.getDocsUpperBound());
     }
@@ -897,20 +894,20 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
       lst.add("soft autocommit maxTime", "" + softCommitTracker.getTimeUpperBound() + "ms");
     }
     lst.add("soft autocommits", softCommitTracker.getCommitCount());
-    lst.add("optimizes", optimizeCommands.get());
-    lst.add("rollbacks", rollbackCommands.get());
-    lst.add("expungeDeletes", expungeDeleteCommands.get());
-    lst.add("docsPending", numDocsPending.get());
+    lst.add("optimizes", optimizeCommands.longValue());
+    lst.add("rollbacks", rollbackCommands.longValue());
+    lst.add("expungeDeletes", expungeDeleteCommands.longValue());
+    lst.add("docsPending", numDocsPending.longValue());
     // pset.size() not synchronized, but it should be fine to access.
     // lst.add("deletesPending", pset.size());
-    lst.add("adds", addCommands.get());
-    lst.add("deletesById", deleteByIdCommands.get());
-    lst.add("deletesByQuery", deleteByQueryCommands.get());
-    lst.add("errors", numErrors.get());
-    lst.add("cumulative_adds", addCommandsCumulative.get());
-    lst.add("cumulative_deletesById", deleteByIdCommandsCumulative.get());
-    lst.add("cumulative_deletesByQuery", deleteByQueryCommandsCumulative.get());
-    lst.add("cumulative_errors", numErrorsCumulative.get());
+    lst.add("adds", addCommands.longValue());
+    lst.add("deletesById", deleteByIdCommands.longValue());
+    lst.add("deletesByQuery", deleteByQueryCommands.longValue());
+    lst.add("errors", numErrors.longValue());
+    lst.add("cumulative_adds", addCommandsCumulative.longValue());
+    lst.add("cumulative_deletesById", deleteByIdCommandsCumulative.longValue());
+    lst.add("cumulative_deletesByQuery", deleteByQueryCommandsCumulative.longValue());
+    lst.add("cumulative_errors", numErrorsCumulative.longValue());
     if (this.ulog != null) {
       lst.add("transaction_logs_total_size", ulog.getTotalLogsSize());
       lst.add("transaction_logs_total_number", ulog.getTotalLogsNumber());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f906862a/solr/core/src/java/org/apache/solr/util/ConcurrentLRUCache.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/ConcurrentLRUCache.java b/solr/core/src/java/org/apache/solr/util/ConcurrentLRUCache.java
index 3b6db53..be14437 100644
--- a/solr/core/src/java/org/apache/solr/util/ConcurrentLRUCache.java
+++ b/solr/core/src/java/org/apache/solr/util/ConcurrentLRUCache.java
@@ -28,6 +28,7 @@ import java.util.TreeSet;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.LongAdder;
 import java.util.concurrent.locks.ReentrantLock;
 import java.lang.invoke.MethodHandles;
 import java.lang.ref.WeakReference;
@@ -90,7 +91,7 @@ public class ConcurrentLRUCache<K,V> implements Cache<K,V> {
   public V get(K key) {
     CacheEntry<K,V> e = map.get(key);
     if (e == null) {
-      if (islive) stats.missCounter.incrementAndGet();
+      if (islive) stats.missCounter.increment();
       return null;
     }
     if (islive) e.lastAccessed = stats.accessCounter.incrementAndGet();
@@ -119,9 +120,9 @@ public class ConcurrentLRUCache<K,V> implements Cache<K,V> {
       currentSize = stats.size.get();
     }
     if (islive) {
-      stats.putCounter.incrementAndGet();
+      stats.putCounter.increment();
     } else {
-      stats.nonLivePutCounter.incrementAndGet();
+      stats.nonLivePutCounter.increment();
     }
 
     // Check if we need to clear out old entries from the cache.
@@ -172,7 +173,7 @@ public class ConcurrentLRUCache<K,V> implements Cache<K,V> {
       isCleaning = true;
       this.oldestEntry = oldestEntry;     // volatile write to make isCleaning visible
 
-      long timeCurrent = stats.accessCounter.get();
+      long timeCurrent = stats.accessCounter.longValue();
       int sz = stats.size.get();
 
       int numRemoved = 0;
@@ -532,23 +533,23 @@ public class ConcurrentLRUCache<K,V> implements Cache<K,V> {
 
 
   public static class Stats {
-    private final AtomicLong accessCounter = new AtomicLong(0),
-            putCounter = new AtomicLong(0),
-            nonLivePutCounter = new AtomicLong(0),
-            missCounter = new AtomicLong();
+    private final AtomicLong accessCounter = new AtomicLong(0);
+    private final LongAdder putCounter = new LongAdder();
+    private final LongAdder nonLivePutCounter = new LongAdder();
+    private final LongAdder missCounter = new LongAdder();
     private final AtomicInteger size = new AtomicInteger();
     private AtomicLong evictionCounter = new AtomicLong();
 
     public long getCumulativeLookups() {
-      return (accessCounter.get() - putCounter.get() - nonLivePutCounter.get()) + missCounter.get();
+      return (accessCounter.longValue() - putCounter.longValue() - nonLivePutCounter.longValue()) + missCounter.longValue();
     }
 
     public long getCumulativeHits() {
-      return accessCounter.get() - putCounter.get() - nonLivePutCounter.get();
+      return accessCounter.longValue() - putCounter.longValue() - nonLivePutCounter.longValue();
     }
 
     public long getCumulativePuts() {
-      return putCounter.get();
+      return putCounter.longValue();
     }
 
     public long getCumulativeEvictions() {
@@ -560,18 +561,18 @@ public class ConcurrentLRUCache<K,V> implements Cache<K,V> {
     }
 
     public long getCumulativeNonLivePuts() {
-      return nonLivePutCounter.get();
+      return nonLivePutCounter.longValue();
     }
 
     public long getCumulativeMisses() {
-      return missCounter.get();
+      return missCounter.longValue();
     }
 
     public void add(Stats other) {
       accessCounter.addAndGet(other.accessCounter.get());
-      putCounter.addAndGet(other.putCounter.get());
-      nonLivePutCounter.addAndGet(other.nonLivePutCounter.get());
-      missCounter.addAndGet(other.missCounter.get());
+      putCounter.add(other.putCounter.longValue());
+      nonLivePutCounter.add(other.nonLivePutCounter.longValue());
+      missCounter.add(other.missCounter.longValue());
       evictionCounter.addAndGet(other.evictionCounter.get());
       size.set(Math.max(size.get(), other.size.get()));
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f906862a/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java b/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java
index 2bde118..ca604fe 100644
--- a/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java
@@ -160,13 +160,13 @@ public class DirectUpdateHandlerTest extends SolrTestCaseJ4 {
     SolrQueryRequest ureq = req();
     CommitUpdateCommand cmtCmd = new CommitUpdateCommand(ureq, false);
     cmtCmd.waitSearcher = true;
-    assertEquals( 1, duh2.addCommands.get() );
-    assertEquals( 1, duh2.addCommandsCumulative.get() );
-    assertEquals( 0, duh2.commitCommands.get() );
+    assertEquals( 1, duh2.addCommands.longValue() );
+    assertEquals( 1, duh2.addCommandsCumulative.longValue() );
+    assertEquals( 0, duh2.commitCommands.longValue() );
     updater.commit(cmtCmd);
-    assertEquals( 0, duh2.addCommands.get() );
-    assertEquals( 1, duh2.addCommandsCumulative.get() );
-    assertEquals( 1, duh2.commitCommands.get() );
+    assertEquals( 0, duh2.addCommands.longValue() );
+    assertEquals( 1, duh2.addCommandsCumulative.longValue() );
+    assertEquals( 1, duh2.commitCommands.longValue() );
     ureq.close();
 
     assertU(adoc("id","B"));
@@ -174,13 +174,13 @@ public class DirectUpdateHandlerTest extends SolrTestCaseJ4 {
     // rollback "B"
     ureq = req();
     RollbackUpdateCommand rbkCmd = new RollbackUpdateCommand(ureq);
-    assertEquals( 1, duh2.addCommands.get() );
-    assertEquals( 2, duh2.addCommandsCumulative.get() );
-    assertEquals( 0, duh2.rollbackCommands.get() );
+    assertEquals( 1, duh2.addCommands.longValue() );
+    assertEquals( 2, duh2.addCommandsCumulative.longValue() );
+    assertEquals( 0, duh2.rollbackCommands.longValue() );
     updater.rollback(rbkCmd);
-    assertEquals( 0, duh2.addCommands.get() );
-    assertEquals( 1, duh2.addCommandsCumulative.get() );
-    assertEquals( 1, duh2.rollbackCommands.get() );
+    assertEquals( 0, duh2.addCommands.longValue() );
+    assertEquals( 1, duh2.addCommandsCumulative.longValue() );
+    assertEquals( 1, duh2.rollbackCommands.longValue() );
     ureq.close();
     
     // search - "B" should not be found.
@@ -220,13 +220,13 @@ public class DirectUpdateHandlerTest extends SolrTestCaseJ4 {
     SolrQueryRequest ureq = req();
     CommitUpdateCommand cmtCmd = new CommitUpdateCommand(ureq, false);
     cmtCmd.waitSearcher = true;
-    assertEquals( 2, duh2.addCommands.get() );
-    assertEquals( 2, duh2.addCommandsCumulative.get() );
-    assertEquals( 0, duh2.commitCommands.get() );
+    assertEquals( 2, duh2.addCommands.longValue() );
+    assertEquals( 2, duh2.addCommandsCumulative.longValue() );
+    assertEquals( 0, duh2.commitCommands.longValue() );
     updater.commit(cmtCmd);
-    assertEquals( 0, duh2.addCommands.get() );
-    assertEquals( 2, duh2.addCommandsCumulative.get() );
-    assertEquals( 1, duh2.commitCommands.get() );
+    assertEquals( 0, duh2.addCommands.longValue() );
+    assertEquals( 2, duh2.addCommandsCumulative.longValue() );
+    assertEquals( 1, duh2.commitCommands.longValue() );
     ureq.close();
 
     // search - "A","B" should be found.
@@ -253,14 +253,14 @@ public class DirectUpdateHandlerTest extends SolrTestCaseJ4 {
     // rollback "B"
     ureq = req();
     RollbackUpdateCommand rbkCmd = new RollbackUpdateCommand(ureq);
-    assertEquals( 1, duh2.deleteByIdCommands.get() );
-    assertEquals( 1, duh2.deleteByIdCommandsCumulative.get() );
-    assertEquals( 0, duh2.rollbackCommands.get() );
+    assertEquals( 1, duh2.deleteByIdCommands.longValue() );
+    assertEquals( 1, duh2.deleteByIdCommandsCumulative.longValue() );
+    assertEquals( 0, duh2.rollbackCommands.longValue() );
     updater.rollback(rbkCmd);
     ureq.close();
-    assertEquals( 0, duh2.deleteByIdCommands.get() );
-    assertEquals( 0, duh2.deleteByIdCommandsCumulative.get() );
-    assertEquals( 1, duh2.rollbackCommands.get() );
+    assertEquals( 0, duh2.deleteByIdCommands.longValue() );
+    assertEquals( 0, duh2.deleteByIdCommandsCumulative.longValue() );
+    assertEquals( 1, duh2.rollbackCommands.longValue() );
     
     // search - "B" should be found.
     assertQ("\"B\" should be found.", req


[16/40] lucene-solr:apiv2: reduce IW.infoStream noise when stalling happens due to too many total bytes flushing; only notifyAll in stall/unstall when it changes

Posted by sa...@apache.org.
reduce IW.infoStream noise when stalling happens due to too many total bytes flushing; only notifyAll in stall/unstall when it changes


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

Branch: refs/heads/apiv2
Commit: 614b45d2dff43cb05df1f4fa65a40d0568bda616
Parents: 3a0a9fd
Author: Mike McCandless <mi...@apache.org>
Authored: Mon Jul 25 19:17:28 2016 -0400
Committer: Mike McCandless <mi...@apache.org>
Committed: Mon Jul 25 19:18:23 2016 -0400

----------------------------------------------------------------------
 .../apache/lucene/index/DocumentsWriter.java    | 11 --------
 .../index/DocumentsWriterFlushControl.java      | 27 ++++++++++++++------
 .../index/DocumentsWriterStallControl.java      | 26 +++++--------------
 .../index/TestDocumentsWriterStallControl.java  |  6 ++---
 4 files changed, 28 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/614b45d2/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java
index 2429c33..2807517 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java
@@ -376,9 +376,6 @@ final class DocumentsWriter implements Closeable, Accountable {
     boolean hasEvents = false;
     if (flushControl.anyStalledThreads() || flushControl.numQueuedFlushes() > 0) {
       // Help out flushing any queued DWPTs so we can un-stall:
-      if (infoStream.isEnabled("DW")) {
-        infoStream.message("DW", "DocumentsWriter has queued dwpt; will hijack this thread to flush pending segment(s)");
-      }
       do {
         // Try pick up pending threads here if possible
         DocumentsWriterPerThread flushingDWPT;
@@ -386,17 +383,9 @@ final class DocumentsWriter implements Closeable, Accountable {
           // Don't push the delete here since the update could fail!
           hasEvents |= doFlush(flushingDWPT);
         }
-  
-        if (infoStream.isEnabled("DW") && flushControl.anyStalledThreads()) {
-          infoStream.message("DW", "WARNING DocumentsWriter has stalled threads; waiting");
-        }
         
         flushControl.waitIfStalled(); // block if stalled
       } while (flushControl.numQueuedFlushes() != 0); // still queued DWPTs try help flushing
-
-      if (infoStream.isEnabled("DW")) {
-        infoStream.message("DW", "continue indexing after helping out flushing DocumentsWriter is healthy");
-      }
     }
     return hasEvents;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/614b45d2/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
index a8c1dc3..a5b4b7c 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
@@ -22,6 +22,7 @@ import java.util.IdentityHashMap;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Locale;
 import java.util.Queue;
 import java.util.concurrent.atomic.AtomicBoolean;
 
@@ -74,7 +75,7 @@ final class DocumentsWriterFlushControl implements Accountable {
 
   DocumentsWriterFlushControl(DocumentsWriter documentsWriter, LiveIndexWriterConfig config, BufferedUpdatesStream bufferedUpdatesStream) {
     this.infoStream = config.getInfoStream();
-    this.stallControl = new DocumentsWriterStallControl(config);
+    this.stallControl = new DocumentsWriterStallControl();
     this.perThreadPool = documentsWriter.perThreadPool;
     this.flushPolicy = documentsWriter.flushPolicy;
     this.config = config;
@@ -230,7 +231,9 @@ final class DocumentsWriterFlushControl implements Accountable {
       }
     }
   }
-  
+
+  private long stallStartNS;
+
   private boolean updateStallState() {
     
     assert Thread.holdsLock(this);
@@ -245,6 +248,20 @@ final class DocumentsWriterFlushControl implements Accountable {
     final boolean stall = (activeBytes + flushBytes) > limit &&
       activeBytes < limit &&
       !closed;
+
+    if (infoStream.isEnabled("DWFC")) {
+      if (stall != stallControl.anyStalledThreads()) {
+        if (stall) {
+          infoStream.message("DW", String.format(Locale.ROOT, "now stalling flushes: netBytes: %.1f MB flushBytes: %.1f MB fullFlush: %b",
+                                                 netBytes()/1024./1024., flushBytes()/1024./1024., fullFlush));
+          stallStartNS = System.nanoTime();
+        } else {
+          infoStream.message("DW", String.format(Locale.ROOT, "done stalling flushes for %.1f msec: netBytes: %.1f MB flushBytes: %.1f MB fullFlush: %b",
+                                                 (System.nanoTime()-stallStartNS)/1000000., netBytes()/1024./1024., flushBytes()/1024./1024., fullFlush));
+        }
+      }
+    }
+
     stallControl.updateStalled(stall);
     return stall;
   }
@@ -687,12 +704,6 @@ final class DocumentsWriterFlushControl implements Accountable {
    * checked out DWPT are available
    */
   void waitIfStalled() {
-    if (infoStream.isEnabled("DWFC")) {
-      infoStream.message("DWFC",
-          "waitIfStalled: numFlushesPending: " + flushQueue.size()
-              + " netBytes: " + netBytes() + " flushBytes: " + flushBytes()
-              + " fullFlush: " + fullFlush);
-    }
     stallControl.waitIfStalled();
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/614b45d2/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterStallControl.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterStallControl.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterStallControl.java
index 84fa9af..c46e3d2 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterStallControl.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterStallControl.java
@@ -20,7 +20,6 @@ import java.util.IdentityHashMap;
 import java.util.Map;
 
 import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState;
-import org.apache.lucene.util.InfoStream;
 import org.apache.lucene.util.ThreadInterruptedException;
 
 /**
@@ -44,12 +43,7 @@ final class DocumentsWriterStallControl {
   private int numWaiting; // only with assert
   private boolean wasStalled; // only with assert
   private final Map<Thread, Boolean> waiting = new IdentityHashMap<>(); // only with assert
-  private final InfoStream infoStream;
 
-  DocumentsWriterStallControl(LiveIndexWriterConfig iwc) {
-    infoStream = iwc.getInfoStream();
-  }
-  
   /**
    * Update the stalled flag status. This method will set the stalled flag to
    * <code>true</code> iff the number of flushing
@@ -59,11 +53,13 @@ final class DocumentsWriterStallControl {
    * waiting on {@link #waitIfStalled()}
    */
   synchronized void updateStalled(boolean stalled) {
-    this.stalled = stalled;
-    if (stalled) {
-      wasStalled = true;
+    if (this.stalled != stalled) {
+      this.stalled = stalled;
+      if (stalled) {
+        wasStalled = true;
+      }
+      notifyAll();
     }
-    notifyAll();
   }
   
   /**
@@ -93,13 +89,7 @@ final class DocumentsWriterStallControl {
     return stalled;
   }
   
-  long stallStartNS;
-
   private void incWaiters() {
-    stallStartNS = System.nanoTime();
-    if (infoStream.isEnabled("DW") && numWaiting == 0) {
-      infoStream.message("DW", "now stalling flushes");
-    }
     numWaiting++;
     assert waiting.put(Thread.currentThread(), Boolean.TRUE) == null;
     assert numWaiting > 0;
@@ -109,10 +99,6 @@ final class DocumentsWriterStallControl {
     numWaiting--;
     assert waiting.remove(Thread.currentThread()) != null;
     assert numWaiting >= 0;
-    if (infoStream.isEnabled("DW") && numWaiting == 0) {
-      long stallEndNS = System.nanoTime();
-      infoStream.message("DW", "done stalling flushes for " + ((stallEndNS - stallStartNS)/1000000.0) + " ms");
-    }
   }
   
   synchronized boolean hasBlocked() { // for tests

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/614b45d2/lucene/core/src/test/org/apache/lucene/index/TestDocumentsWriterStallControl.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDocumentsWriterStallControl.java b/lucene/core/src/test/org/apache/lucene/index/TestDocumentsWriterStallControl.java
index 2575e61..d5e643a 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestDocumentsWriterStallControl.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestDocumentsWriterStallControl.java
@@ -32,7 +32,7 @@ import org.apache.lucene.util.ThreadInterruptedException;
 public class TestDocumentsWriterStallControl extends LuceneTestCase {
   
   public void testSimpleStall() throws InterruptedException {
-    DocumentsWriterStallControl ctrl = new DocumentsWriterStallControl(newIndexWriterConfig());
+    DocumentsWriterStallControl ctrl = new DocumentsWriterStallControl();
    
     ctrl.updateStalled(false);
     Thread[] waitThreads = waitThreads(atLeast(1), ctrl);
@@ -54,7 +54,7 @@ public class TestDocumentsWriterStallControl extends LuceneTestCase {
   }
   
   public void testRandom() throws InterruptedException {
-    final DocumentsWriterStallControl ctrl = new DocumentsWriterStallControl(newIndexWriterConfig());
+    final DocumentsWriterStallControl ctrl = new DocumentsWriterStallControl();
     ctrl.updateStalled(false);
     
     Thread[] stallThreads = new Thread[atLeast(3)];
@@ -95,7 +95,7 @@ public class TestDocumentsWriterStallControl extends LuceneTestCase {
   }
   
   public void testAccquireReleaseRace() throws InterruptedException {
-    final DocumentsWriterStallControl ctrl = new DocumentsWriterStallControl(newIndexWriterConfig());
+    final DocumentsWriterStallControl ctrl = new DocumentsWriterStallControl();
     ctrl.updateStalled(false);
     final AtomicBoolean stop = new AtomicBoolean(false);
     final AtomicBoolean checkPoint = new AtomicBoolean(true);


[13/40] lucene-solr:apiv2: LUCENE-7391: Fix MemoryIndex fields() perf regression

Posted by sa...@apache.org.
LUCENE-7391: Fix MemoryIndex fields() perf regression


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

Branch: refs/heads/apiv2
Commit: dcc9a4b9a411a0436e5cb21e7d6251691640f3db
Parents: 2d07ffd
Author: David Smiley <ds...@apache.org>
Authored: Mon Jul 25 09:55:32 2016 -0400
Committer: David Smiley <ds...@apache.org>
Committed: Mon Jul 25 09:59:00 2016 -0400

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  3 +++
 .../apache/lucene/index/memory/MemoryIndex.java | 24 ++++++++++----------
 .../lucene/index/memory/TestMemoryIndex.java    | 14 ++++++++++++
 3 files changed, 29 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dcc9a4b9/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 1edac7e..7b53017 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -79,6 +79,9 @@ Bug Fixes
 * LUCENE-7389: Fix FieldType.setDimensions(...) validation for the dimensionNumBytes
   parameter. (Martijn van Groningen)
 
+* LUCENE-7391: Fix performance regression in MemoryIndex's fields() introduced
+  in Lucene 6. (Steve Mason via David Smiley)
+
 Improvements
 
 * LUCENE-7323: Compound file writing now verifies the incoming

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dcc9a4b9/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
----------------------------------------------------------------------
diff --git a/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java b/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
index cdd53ed..9037dfa 100644
--- a/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
+++ b/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
@@ -21,11 +21,9 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Iterator;
-import java.util.Locale;
 import java.util.Map;
 import java.util.SortedMap;
 import java.util.TreeMap;
-import java.util.stream.Collectors;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
@@ -1045,6 +1043,7 @@ public class MemoryIndex {
   private final class MemoryIndexReader extends LeafReader {
 
     private final PointValues pointValues;
+    private Fields memoryFields = new MemoryFields(fields);
 
     private MemoryIndexReader() {
       super(); // avoid as much superclass baggage as possible
@@ -1198,13 +1197,7 @@ public class MemoryIndex {
 
     @Override
     public Fields fields() {
-      Map<String, Info> filteredFields = fields.entrySet().stream()
-          .filter(entry ->  entry.getValue().numTokens > 0)
-          .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue,
-              (u,v) -> { throw new IllegalStateException(String.format(Locale.ROOT, "Duplicate key %s", u));},
-              TreeMap::new
-          ));
-      return new MemoryFields(filteredFields );
+      return memoryFields;
     }
 
     private class MemoryFields extends Fields {
@@ -1217,13 +1210,16 @@ public class MemoryIndex {
 
       @Override
       public Iterator<String> iterator() {
-        return fields.keySet().iterator();
+        return fields.entrySet().stream()
+            .filter(e -> e.getValue().numTokens > 0)
+            .map(Map.Entry::getKey)
+            .iterator();
       }
 
       @Override
       public Terms terms(final String field) {
         final Info info = fields.get(field);
-        if (info == null) {
+        if (info == null || info.numTokens <= 0) {
           return null;
         }
 
@@ -1278,7 +1274,11 @@ public class MemoryIndex {
 
       @Override
       public int size() {
-        return fields.size();
+        int size = 0;
+        for (String fieldName : this) {
+          size++;
+        }
+        return size;
       }
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dcc9a4b9/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java
----------------------------------------------------------------------
diff --git a/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java b/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java
index 2f95a4e..1d7c60e 100644
--- a/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java
+++ b/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java
@@ -45,6 +45,7 @@ import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.index.FieldInvertState;
+import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.NumericDocValues;
@@ -129,6 +130,19 @@ public class TestMemoryIndex extends LuceneTestCase {
     assertEquals("be", terms.term().utf8ToString());
     TestUtil.checkReader(reader);
   }
+
+  public void testFieldsOnlyReturnsIndexedFields() throws IOException {
+    Document doc = new Document();
+
+    doc.add(new NumericDocValuesField("numeric", 29L));
+    doc.add(new TextField("text", "some text", Field.Store.NO));
+
+    MemoryIndex mi = MemoryIndex.fromDocument(doc, analyzer);
+    IndexSearcher searcher = mi.createSearcher();
+    IndexReader reader = searcher.getIndexReader();
+
+    assertEquals(reader.getTermVectors(0).size(), 1);
+  }
   
   public void testReaderConsistency() throws IOException {
     Analyzer analyzer = new MockPayloadAnalyzer();