You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ds...@apache.org on 2016/04/27 20:21:45 UTC

[01/50] [abbrv] lucene-solr:solr-5750: SOLR-8913: When using a shared filesystem we should store data dir and tlog dir locations in the cluster state.

Repository: lucene-solr
Updated Branches:
  refs/heads/solr-5750 e8d296864 -> ca173ab1a


SOLR-8913: When using a shared filesystem we should store data dir and tlog dir locations in the cluster state.


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

Branch: refs/heads/solr-5750
Commit: b44ca080d6967fce9f6689e38f2b52968bb96d81
Parents: 221ecfa
Author: markrmiller <ma...@apache.org>
Authored: Wed Apr 20 11:54:09 2016 -0400
Committer: markrmiller <ma...@apache.org>
Committed: Wed Apr 20 12:05:15 2016 -0400

----------------------------------------------------------------------
 solr/CHANGES.txt                                   |  3 +++
 .../java/org/apache/solr/cloud/ZkController.java   |  5 ++---
 .../org/apache/solr/cloud/hdfs/StressHdfsTest.java | 17 ++++++++++++++++-
 3 files changed, 21 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b44ca080/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 9e184e3..8630ea4 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -103,6 +103,9 @@ New Features
 
 * SOLR-8918: Adds Streaming to the admin page under the collections section. Includes
   ability to see graphically the expression explanation (Dennis Gove)
+  
+* SOLR-8913: When using a shared filesystem we should store data dir and tlog dir locations in 
+  the cluster state. (Mark Miller)
 
 * SOLR-8809: Implement Connection.prepareStatement (Kevin Risden)
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b44ca080/solr/core/src/java/org/apache/solr/cloud/ZkController.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkController.java b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
index bdd6a62..3afc3b9 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
@@ -1165,9 +1165,8 @@ public final class ZkController {
       if (coreNodeName != null) {
         props.put(ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName);
       }
-      
-      if (ClusterStateUtil.isAutoAddReplicas(getZkStateReader(), collection)) {
-        try (SolrCore core = cc.getCore(cd.getName())) {
+      try (SolrCore core = cc.getCore(cd.getName())) {
+        if (core != null && core.getDirectoryFactory().isSharedStorage()) {
           if (core != null && core.getDirectoryFactory().isSharedStorage()) {
             props.put("dataDir", core.getDataDir());
             UpdateLog ulog = core.getUpdateHandler().getUpdateLog();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b44ca080/solr/core/src/test/org/apache/solr/cloud/hdfs/StressHdfsTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/hdfs/StressHdfsTest.java b/solr/core/src/test/org/apache/solr/cloud/hdfs/StressHdfsTest.java
index aeb4cb7..b2dba10 100644
--- a/solr/core/src/test/org/apache/solr/cloud/hdfs/StressHdfsTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/hdfs/StressHdfsTest.java
@@ -32,6 +32,9 @@ import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.cloud.BasicDistributedZkTest;
 import org.apache.solr.cloud.ChaosMonkey;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.params.CollectionParams.CollectionAction;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
@@ -46,6 +49,7 @@ import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 import java.util.Timer;
 import java.util.TimerTask;
@@ -106,7 +110,7 @@ public class StressHdfsTest extends BasicDistributedZkTest {
         createCollection(DELETE_DATA_DIR_COLLECTION, 1, 1, 1);
         
         waitForRecoveriesToFinish(DELETE_DATA_DIR_COLLECTION, false);
-        
+
         ChaosMonkey.stop(jettys.get(0));
         
         // enter safe mode and restart a node
@@ -153,6 +157,17 @@ public class StressHdfsTest extends BasicDistributedZkTest {
     createCollection(DELETE_DATA_DIR_COLLECTION, nShards, rep, maxReplicasPerNode);
 
     waitForRecoveriesToFinish(DELETE_DATA_DIR_COLLECTION, false);
+    
+    // data dirs should be in zk, SOLR-8913
+    ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
+    Slice slice = clusterState.getSlice(DELETE_DATA_DIR_COLLECTION, "shard1");
+    assertNotNull(clusterState.getSlices(DELETE_DATA_DIR_COLLECTION).toString(), slice);
+    Collection<Replica> replicas = slice.getReplicas();
+    for (Replica replica : replicas) {
+      assertNotNull(replica.getProperties().toString(), replica.get("dataDir"));
+      assertNotNull(replica.getProperties().toString(), replica.get("ulogDir"));
+    }
+    
     cloudClient.setDefaultCollection(DELETE_DATA_DIR_COLLECTION);
     cloudClient.getZkStateReader().forceUpdateCollection(DELETE_DATA_DIR_COLLECTION);
     


[32/50] [abbrv] lucene-solr:solr-5750: LUCENE-7254: (sandbox/ only) Don't let abuse cases slow down spatial queries

Posted by ds...@apache.org.
LUCENE-7254: (sandbox/ only) Don't let abuse cases slow down spatial queries


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

Branch: refs/heads/solr-5750
Commit: 6fa5166e41652fc58a5f18db4796e230b1354dbd
Parents: 89c65af
Author: Robert Muir <rm...@apache.org>
Authored: Tue Apr 26 09:16:27 2016 -0400
Committer: Robert Muir <rm...@apache.org>
Committed: Tue Apr 26 09:17:21 2016 -0400

----------------------------------------------------------------------
 .../org/apache/lucene/document/LatLonPoint.java |   3 +-
 .../lucene/document/LatLonPointBoxQuery.java    | 287 +++++++++++++++++++
 .../document/LatLonPointDistanceQuery.java      |  17 +-
 .../document/LatLonPointInPolygonQuery.java     |  18 +-
 .../apache/lucene/document/MatchingPoints.java  |  90 ++++++
 5 files changed, 382 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6fa5166e/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java
index 426a702..a63e4bd 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java
@@ -34,7 +34,6 @@ import org.apache.lucene.search.FieldDoc;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.MatchNoDocsQuery;
-import org.apache.lucene.search.PointRangeQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.TopFieldDocs;
@@ -229,7 +228,7 @@ public class LatLonPoint extends Field {
   }
   
   private static Query newBoxInternal(String field, byte[] min, byte[] max) {
-    return new PointRangeQuery(field, min, max, 2) {
+    return new LatLonPointBoxQuery(field, min, max, 2) {
       @Override
       protected String toString(int dimension, byte[] value) {
         if (dimension == 0) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6fa5166e/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointBoxQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointBoxQuery.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointBoxQuery.java
new file mode 100644
index 0000000..423af05
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointBoxQuery.java
@@ -0,0 +1,287 @@
+/*
+ * 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.PointValues;
+import org.apache.lucene.index.PointValues.IntersectVisitor;
+import org.apache.lucene.index.PointValues.Relation;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.PointRangeQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.util.StringHelper;
+
+/** 
+ * Fast version of {@link PointRangeQuery}. It is fast for actual range queries!
+ * @lucene.experimental
+ */
+abstract class LatLonPointBoxQuery extends Query {
+  final String field;
+  final int numDims;
+  final int bytesPerDim;
+  final byte[] lowerPoint;
+  final byte[] upperPoint;
+
+  /** 
+   * Expert: create a multidimensional range query for point values.
+   *
+   * @param field field name. must not be {@code null}.
+   * @param lowerPoint lower portion of the range (inclusive).
+   * @param upperPoint upper portion of the range (inclusive).
+   * @param numDims number of dimensions.
+   * @throws IllegalArgumentException if {@code field} is null, or if {@code lowerValue.length != upperValue.length}
+   */
+  protected LatLonPointBoxQuery(String field, byte[] lowerPoint, byte[] upperPoint, int numDims) {
+    checkArgs(field, lowerPoint, upperPoint);
+    this.field = field;
+    if (numDims <= 0) {
+      throw new IllegalArgumentException("numDims must be positive, got " + numDims);
+    }
+    if (lowerPoint.length == 0) {
+      throw new IllegalArgumentException("lowerPoint has length of zero");
+    }
+    if (lowerPoint.length % numDims != 0) {
+      throw new IllegalArgumentException("lowerPoint is not a fixed multiple of numDims");
+    }
+    if (lowerPoint.length != upperPoint.length) {
+      throw new IllegalArgumentException("lowerPoint has length=" + lowerPoint.length + " but upperPoint has different length=" + upperPoint.length);
+    }
+    this.numDims = numDims;
+    this.bytesPerDim = lowerPoint.length / numDims;
+
+    this.lowerPoint = lowerPoint;
+    this.upperPoint = upperPoint;
+  }
+
+  /** 
+   * Check preconditions for all factory methods
+   * @throws IllegalArgumentException if {@code field}, {@code lowerPoint} or {@code upperPoint} are null.
+   */
+  public static void checkArgs(String field, Object lowerPoint, Object upperPoint) {
+    if (field == null) {
+      throw new IllegalArgumentException("field must not be null");
+    }
+    if (lowerPoint == null) {
+      throw new IllegalArgumentException("lowerPoint must not be null");
+    }
+    if (upperPoint == null) {
+      throw new IllegalArgumentException("upperPoint must not be null");
+    }
+  }
+
+  @Override
+  public final Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
+
+    // We don't use RandomAccessWeight here: it's no good to approximate with "match all docs".
+    // This is an inverted structure and should be used in the first pass:
+
+    return new ConstantScoreWeight(this) {
+
+      private DocIdSetIterator buildMatchingIterator(LeafReader reader, PointValues values) throws IOException {
+        MatchingPoints result = new MatchingPoints(reader, field);
+
+        values.intersect(field,
+            new IntersectVisitor() {
+
+              @Override
+              public void visit(int docID) {
+                result.add(docID);
+              }
+
+              @Override
+              public void visit(int docID, byte[] packedValue) {
+                for(int dim=0;dim<numDims;dim++) {
+                  int offset = dim*bytesPerDim;
+                  if (StringHelper.compare(bytesPerDim, packedValue, offset, lowerPoint, offset) < 0) {
+                    // Doc's value is too low, in this dimension
+                    return;
+                  }
+                  if (StringHelper.compare(bytesPerDim, packedValue, offset, upperPoint, offset) > 0) {
+                    // Doc's value is too high, in this dimension
+                    return;
+                  }
+                }
+
+                // Doc is in-bounds
+                result.add(docID);
+              }
+
+              @Override
+              public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+
+                boolean crosses = false;
+
+                for(int dim=0;dim<numDims;dim++) {
+                  int offset = dim*bytesPerDim;
+
+                  if (StringHelper.compare(bytesPerDim, minPackedValue, offset, upperPoint, offset) > 0 ||
+                      StringHelper.compare(bytesPerDim, maxPackedValue, offset, lowerPoint, offset) < 0) {
+                    return Relation.CELL_OUTSIDE_QUERY;
+                  }
+
+                  crosses |= StringHelper.compare(bytesPerDim, minPackedValue, offset, lowerPoint, offset) < 0 ||
+                    StringHelper.compare(bytesPerDim, maxPackedValue, offset, upperPoint, offset) > 0;
+                }
+
+                if (crosses) {
+                  return Relation.CELL_CROSSES_QUERY;
+                } else {
+                  return Relation.CELL_INSIDE_QUERY;
+                }
+              }
+            });
+        return result.iterator();
+      }
+
+      @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 points
+          return null;
+        }
+        FieldInfo fieldInfo = reader.getFieldInfos().fieldInfo(field);
+        if (fieldInfo == null) {
+          // No docs in this segment indexed this field at all
+          return null;
+        }
+        if (fieldInfo.getPointDimensionCount() != numDims) {
+          throw new IllegalArgumentException("field=\"" + field + "\" was indexed with numDims=" + fieldInfo.getPointDimensionCount() + " but this query has numDims=" + numDims);
+        }
+        if (bytesPerDim != fieldInfo.getPointNumBytes()) {
+          throw new IllegalArgumentException("field=\"" + field + "\" was indexed with bytesPerDim=" + fieldInfo.getPointNumBytes() + " but this query has bytesPerDim=" + bytesPerDim);
+        }
+
+        boolean allDocsMatch;
+        if (values.getDocCount(field) == reader.maxDoc()) {
+          final byte[] fieldPackedLower = values.getMinPackedValue(field);
+          final byte[] fieldPackedUpper = values.getMaxPackedValue(field);
+          allDocsMatch = true;
+          for (int i = 0; i < numDims; ++i) {
+            int offset = i * bytesPerDim;
+            if (StringHelper.compare(bytesPerDim, lowerPoint, offset, fieldPackedLower, offset) > 0
+                || StringHelper.compare(bytesPerDim, upperPoint, offset, fieldPackedUpper, offset) < 0) {
+              allDocsMatch = false;
+              break;
+            }
+          }
+        } else {
+          allDocsMatch = false;
+        }
+
+        DocIdSetIterator iterator;
+        if (allDocsMatch) {
+          // all docs have a value and all points are within bounds, so everything matches
+          iterator = DocIdSetIterator.all(reader.maxDoc());
+        } else {
+          iterator = buildMatchingIterator(reader, values);
+        }
+
+        return new ConstantScoreScorer(this, score(), iterator);
+      }
+    };
+  }
+
+  @Override
+  public final int hashCode() {
+    int hash = super.hashCode();
+    hash = 31 * hash + field.hashCode();
+    hash = 31 * hash + Arrays.hashCode(lowerPoint);
+    hash = 31 * hash + Arrays.hashCode(upperPoint);
+    hash = 31 * hash + numDims;
+    hash = 31 * hash + Objects.hashCode(bytesPerDim);
+    return hash;
+  }
+
+  @Override
+  public final boolean equals(Object other) {
+    if (super.equals(other) == false) {
+      return false;
+    }
+
+    final LatLonPointBoxQuery q = (LatLonPointBoxQuery) other;
+    if (field.equals(q.field) == false) {
+      return false;
+    }
+
+    if (q.numDims != numDims) {
+      return false;
+    }
+
+    if (q.bytesPerDim != bytesPerDim) {
+      return false;
+    }
+
+    if (Arrays.equals(lowerPoint, q.lowerPoint) == false) {
+      return false;
+    }
+    
+    if (Arrays.equals(upperPoint, q.upperPoint) == false) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public final String toString(String field) {
+    final StringBuilder sb = new StringBuilder();
+    if (this.field.equals(field) == false) {
+      sb.append(this.field);
+      sb.append(':');
+    }
+
+    // print ourselves as "range per dimension"
+    for (int i = 0; i < numDims; i++) {
+      if (i > 0) {
+        sb.append(',');
+      }
+      
+      int startOffset = bytesPerDim * i;
+
+      sb.append('[');
+      sb.append(toString(i, Arrays.copyOfRange(lowerPoint, startOffset, startOffset + bytesPerDim)));
+      sb.append(" TO ");
+      sb.append(toString(i, Arrays.copyOfRange(upperPoint, startOffset, startOffset + bytesPerDim)));
+      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 value single value, never null
+   * @return human readable value for debugging
+   */
+  protected abstract String toString(int dimension, byte[] value);
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6fa5166e/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointDistanceQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointDistanceQuery.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointDistanceQuery.java
index 9bd78fe..0759ce1 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointDistanceQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointDistanceQuery.java
@@ -28,13 +28,10 @@ import org.apache.lucene.index.PointValues.IntersectVisitor;
 import org.apache.lucene.index.PointValues.Relation;
 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.NumericUtils;
 import org.apache.lucene.util.SloppyMath;
 import org.apache.lucene.util.StringHelper;
@@ -120,16 +117,11 @@ final class LatLonPointDistanceQuery extends Query {
         LatLonPoint.checkCompatible(fieldInfo);
         
         // matching docids
-        DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc());
+        MatchingPoints result = new MatchingPoints(reader, field);
 
         values.intersect(field,
                          new IntersectVisitor() {
                            @Override
-                           public void grow(int count) {
-                             result.grow(count);
-                           }
-
-                           @Override
                            public void visit(int docID) {
                              result.add(docID);
                            }
@@ -209,12 +201,7 @@ final class LatLonPointDistanceQuery extends Query {
                            }
                          });
 
-        DocIdSet set = result.build();
-        final DocIdSetIterator disi = set.iterator();
-        if (disi == null) {
-          return null;
-        }
-        return new ConstantScoreScorer(this, score(), disi);
+        return new ConstantScoreScorer(this, score(), result.iterator());
       }
     };
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6fa5166e/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java
index ee7c1e8..506e6b9 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java
@@ -24,8 +24,6 @@ import org.apache.lucene.index.PointValues.IntersectVisitor;
 import org.apache.lucene.index.PointValues.Relation;
 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;
@@ -34,7 +32,6 @@ import org.apache.lucene.index.PointValues;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.util.DocIdSetBuilder;
 import org.apache.lucene.util.NumericUtils;
 import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.geo.Polygon;
@@ -113,16 +110,11 @@ final class LatLonPointInPolygonQuery extends Query {
         LatLonPoint.checkCompatible(fieldInfo);
 
         // matching docids
-        DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc());
+        MatchingPoints result = new MatchingPoints(reader, field);
 
         values.intersect(field, 
                          new IntersectVisitor() {
                            @Override
-                           public void grow(int count) {
-                             result.grow(count);
-                           }
-
-                           @Override
                            public void visit(int docID) {
                              result.add(docID);
                            }
@@ -154,13 +146,7 @@ final class LatLonPointInPolygonQuery extends Query {
                            }
                          });
 
-        DocIdSet set = result.build();
-        final DocIdSetIterator disi = set.iterator();
-        if (disi == null) {
-          return null;
-        }
-
-        return new ConstantScoreScorer(this, score(), disi);
+        return new ConstantScoreScorer(this, score(), result.iterator());
       }
     };
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6fa5166e/lucene/sandbox/src/java/org/apache/lucene/document/MatchingPoints.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/MatchingPoints.java b/lucene/sandbox/src/java/org/apache/lucene/document/MatchingPoints.java
new file mode 100644
index 0000000..2b6c124
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/MatchingPoints.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.document;
+
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.BitSet;
+import org.apache.lucene.util.BitSetIterator;
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.SparseFixedBitSet;
+
+/**
+ * Accumulates matching hits for points.
+ * <p>
+ * Add matches with ({@link #add(int)}) and call {@link #iterator()} for
+ * an iterator over the results. 
+ * <p>
+ * This implementation currently optimizes bitset structure (sparse vs dense)
+ * and {@link DocIdSetIterator#cost()} (cardinality) based on index statistics.
+ * This API may change as point values evolves.
+ * 
+ * @lucene.experimental
+ */
+final class MatchingPoints {
+  /** bitset we collect into */
+  private final BitSet bits;
+  /** number of documents containing a value for the points field */
+  private final int docCount;
+  /** number of values indexed for the points field */
+  private final long numPoints;
+  /** number of documents in the index segment */
+  private final int maxDoc;
+  /** counter of hits seen */
+  private long counter;
+
+  /**
+   * Creates a new accumulator.
+   * @param reader reader to collect point matches from
+   * @param field field name.
+   */
+  public MatchingPoints(LeafReader reader, String field) {
+    maxDoc = reader.maxDoc();
+    PointValues values = reader.getPointValues();
+    if (values == null) {
+      throw new IllegalStateException("the query is missing null checks");
+    }
+    docCount = values.getDocCount(field);
+    numPoints = values.size(field);
+    // heuristic: if the field is really sparse, use a sparse impl
+    if (docCount >= 0 && docCount * 100L < maxDoc) {
+      bits = new SparseFixedBitSet(maxDoc);
+    } else {
+      bits = new FixedBitSet(maxDoc);
+    }
+  }
+
+  /**
+   * Record a matching docid.
+   * <p>
+   * NOTE: doc IDs do not need to be provided in any order.
+   */
+  public void add(int doc) {
+    bits.set(doc);
+    counter++;
+  }
+  
+  /**
+   * Returns an iterator over the recorded matches.
+   */
+  public DocIdSetIterator iterator() {
+    // if single-valued (docCount == numPoints), then this is exact
+    // otherwise its approximate based on field stats
+    return new BitSetIterator(bits, (long) (counter * (docCount / (double) numPoints)));
+  }
+}


[18/50] [abbrv] lucene-solr:solr-5750: LUCENE-7249: LatLonPoint polygon should use tree relate()

Posted by ds...@apache.org.
LUCENE-7249: LatLonPoint polygon should use tree relate()


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

Branch: refs/heads/solr-5750
Commit: 88c9da6c899c7015f6c9a818a4a4f91984022254
Parents: 666472b
Author: Robert Muir <rm...@apache.org>
Authored: Fri Apr 22 15:10:39 2016 -0400
Committer: Robert Muir <rm...@apache.org>
Committed: Fri Apr 22 15:11:03 2016 -0400

----------------------------------------------------------------------
 .../src/java/org/apache/lucene/document/LatLonGrid.java        | 4 ++--
 .../org/apache/lucene/document/LatLonPointInPolygonQuery.java  | 5 +++--
 .../src/test/org/apache/lucene/document/TestLatLonGrid.java    | 6 +++---
 3 files changed, 8 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/88c9da6c/lucene/sandbox/src/java/org/apache/lucene/document/LatLonGrid.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonGrid.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonGrid.java
index 2083f03..4b3b2b2 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonGrid.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonGrid.java
@@ -62,12 +62,12 @@ final class LatLonGrid {
   
   final LatLonTree[] tree;
   
-  LatLonGrid(int minLat, int maxLat, int minLon, int maxLon, Polygon... polygons) {
+  LatLonGrid(int minLat, int maxLat, int minLon, int maxLon, LatLonTree[] tree) {
     this.minLat = minLat;
     this.maxLat = maxLat;
     this.minLon = minLon;
     this.maxLon = maxLon;
-    this.tree = LatLonTree.build(polygons);
+    this.tree = tree;
     if (minLon > maxLon) {
       // maybe make 2 grids if you want this? 
       throw new IllegalArgumentException("Grid cannot cross the dateline");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/88c9da6c/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java
index 15361b5..23a98d2 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java
@@ -92,10 +92,11 @@ final class LatLonPointInPolygonQuery extends Query {
     NumericUtils.intToSortableBytes(encodeLongitude(box.minLon), minLon, 0);
     NumericUtils.intToSortableBytes(encodeLongitude(box.maxLon), maxLon, 0);
 
+    final LatLonTree[] tree = LatLonTree.build(polygons);
     final LatLonGrid grid = new LatLonGrid(encodeLatitude(box.minLat),
                                            encodeLatitude(box.maxLat),
                                            encodeLongitude(box.minLon),
-                                           encodeLongitude(box.maxLon), polygons);
+                                           encodeLongitude(box.maxLon), tree);
 
     return new ConstantScoreWeight(this) {
 
@@ -156,7 +157,7 @@ final class LatLonPointInPolygonQuery extends Query {
                              double cellMaxLat = decodeLatitude(maxPackedValue, 0);
                              double cellMaxLon = decodeLongitude(maxPackedValue, Integer.BYTES);
 
-                             return Polygon.relate(polygons, cellMinLat, cellMaxLat, cellMinLon, cellMaxLon);
+                             return LatLonTree.relate(tree, cellMinLat, cellMaxLat, cellMinLon, cellMaxLon);
                            }
                          });
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/88c9da6c/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonGrid.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonGrid.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonGrid.java
index 891e3d5..0c185ea 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonGrid.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonGrid.java
@@ -40,7 +40,7 @@ public class TestLatLonGrid extends LuceneTestCase {
       int maxLat = encodeLatitude(box.maxLat);
       int minLon = encodeLongitude(box.minLon);
       int maxLon = encodeLongitude(box.maxLon);
-      LatLonGrid grid = new LatLonGrid(minLat, maxLat, minLon, maxLon, polygon);
+      LatLonGrid grid = new LatLonGrid(minLat, maxLat, minLon, maxLon, LatLonTree.build(polygon));
       // we are in integer space... but exhaustive testing is slow!
       for (int j = 0; j < 10000; j++) {
         int lat = TestUtil.nextInt(random(), minLat, maxLat);
@@ -79,7 +79,7 @@ public class TestLatLonGrid extends LuceneTestCase {
       int maxLat = encodeLatitude(box.maxLat);
       int minLon = encodeLongitude(box.minLon);
       int maxLon = encodeLongitude(box.maxLon);
-      LatLonGrid grid = new LatLonGrid(minLat, maxLat, minLon, maxLon, polygon);
+      LatLonGrid grid = new LatLonGrid(minLat, maxLat, minLon, maxLon, LatLonTree.build(polygon));
       // we are in integer space... but exhaustive testing is slow!
       for (int j = 0; j < 1000; j++) {
         int lat = TestUtil.nextInt(random(), minLat, maxLat);
@@ -99,7 +99,7 @@ public class TestLatLonGrid extends LuceneTestCase {
     double ONE = decodeLatitude(1);
     Polygon tiny = new Polygon(new double[] { ZERO, ZERO, ONE, ONE, ZERO }, new double[] { ZERO, ONE, ONE, ZERO, ZERO });
     for (int max = 1; max < 500000; max++) {
-      LatLonGrid grid = new LatLonGrid(0, max, 0, max, tiny);
+      LatLonGrid grid = new LatLonGrid(0, max, 0, max, LatLonTree.build(tiny));
       assertEquals(tiny.contains(decodeLatitude(max), decodeLongitude(max)), grid.contains(max, max));
     }
   }


[39/50] [abbrv] lucene-solr:solr-5750: SOLR-8925: Integrate the GraphTermsQuery into the gatherNodes expression

Posted by ds...@apache.org.
SOLR-8925: Integrate the GraphTermsQuery into the gatherNodes expression


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

Branch: refs/heads/solr-5750
Commit: 9ce830d8f2a547b763999ea3790bab6a4d8727a3
Parents: d66f551
Author: jbernste <jb...@apache.org>
Authored: Tue Apr 26 14:25:35 2016 -0400
Committer: jbernste <jb...@apache.org>
Committed: Tue Apr 26 16:33:26 2016 -0400

----------------------------------------------------------------------
 .../solr/search/GraphTermsQParserPlugin.java    |  2 +-
 .../search/TestGraphTermsQParserPlugin.java     |  2 +-
 .../solrj/io/graph/GatherNodesStream.java       | 43 ++++++++++++++------
 .../solrj/io/graph/GraphExpressionTest.java     | 17 ++++++++
 4 files changed, 50 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9ce830d8/solr/core/src/java/org/apache/solr/search/GraphTermsQParserPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/GraphTermsQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/GraphTermsQParserPlugin.java
index 5a2fe11..9c1c162 100644
--- a/solr/core/src/java/org/apache/solr/search/GraphTermsQParserPlugin.java
+++ b/solr/core/src/java/org/apache/solr/search/GraphTermsQParserPlugin.java
@@ -133,7 +133,7 @@ public class GraphTermsQParserPlugin extends QParserPlugin {
         collectTermContext(reader, contexts, termContexts, this.queryTerms);
         for(int i=0; i<termContexts.length; i++) {
           TermContext termContext = termContexts[i];
-          if(termContext != null && termContext.docFreq() < this.maxDocFreq) {
+          if(termContext != null && termContext.docFreq() <= this.maxDocFreq) {
             this.finalContexts.add(termContext);
             this.finalTerms.add(queryTerms[i]);
           }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9ce830d8/solr/core/src/test/org/apache/solr/search/TestGraphTermsQParserPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestGraphTermsQParserPlugin.java b/solr/core/src/test/org/apache/solr/search/TestGraphTermsQParserPlugin.java
index cfff980..858800a 100644
--- a/solr/core/src/test/org/apache/solr/search/TestGraphTermsQParserPlugin.java
+++ b/solr/core/src/test/org/apache/solr/search/TestGraphTermsQParserPlugin.java
@@ -125,7 +125,7 @@ public class TestGraphTermsQParserPlugin extends SolrTestCaseJ4 {
 
     //Test with int field
     params = new ModifiableSolrParams();
-    params.add("q", "{!graphTerms f=test_ti maxDocFreq=3}5,10");
+    params.add("q", "{!graphTerms f=test_ti maxDocFreq=2}5,10");
     params.add("sort", "id asc");
     assertQ(req(params, "indent", "on"), "*[count(//doc)=2]",
         "//result/doc[1]/float[@name='id'][.='6.0']",

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9ce830d8/solr/solrj/src/java/org/apache/solr/client/solrj/io/graph/GatherNodesStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/graph/GatherNodesStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/graph/GatherNodesStream.java
index 7ab6e97..ee76937 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/graph/GatherNodesStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/graph/GatherNodesStream.java
@@ -66,6 +66,7 @@ public class GatherNodesStream extends TupleStream implements Expressible {
   private Iterator<Tuple> out;
   private Traversal traversal;
   private List<Metric> metrics;
+  private int maxDocFreq;
 
   public GatherNodesStream(String zkHost,
                            String collection,
@@ -76,7 +77,8 @@ public class GatherNodesStream extends TupleStream implements Expressible {
                            Map queryParams,
                            List<Metric> metrics,
                            boolean trackTraversal,
-                           Set<Traversal.Scatter> scatter) {
+                           Set<Traversal.Scatter> scatter,
+                           int maxDocFreq) {
 
     init(zkHost,
         collection,
@@ -87,7 +89,8 @@ public class GatherNodesStream extends TupleStream implements Expressible {
         queryParams,
         metrics,
         trackTraversal,
-        scatter);
+        scatter,
+        maxDocFreq);
   }
 
   public GatherNodesStream(StreamExpression expression, StreamFactory factory) throws IOException {
@@ -185,10 +188,11 @@ public class GatherNodesStream extends TupleStream implements Expressible {
       useDefaultTraversal = true;
     }
 
-    StreamExpressionNamedParameter scopeExpression = factory.getNamedOperand(expression, "localScope");
+    StreamExpressionNamedParameter docFreqExpression = factory.getNamedOperand(expression, "maxDocFreq");
+    int docFreq = -1;
 
-    if(trackExpression != null) {
-      trackTraversal = Boolean.parseBoolean(((StreamExpressionValue) trackExpression.getParameter()).getValue());
+    if(docFreqExpression != null) {
+      docFreq = Integer.parseInt(((StreamExpressionValue) docFreqExpression.getParameter()).getValue());
     }
 
     Map<String,String> params = new HashMap<String,String>();
@@ -197,6 +201,7 @@ public class GatherNodesStream extends TupleStream implements Expressible {
           !namedParam.getName().equals("gather") &&
           !namedParam.getName().equals("walk") &&
           !namedParam.getName().equals("scatter") &&
+          !namedParam.getName().equals("maxDocFreq") &&
           !namedParam.getName().equals("trackTraversal"))
       {
         params.put(namedParam.getName(), namedParam.getParameter().toString().trim());
@@ -228,7 +233,8 @@ public class GatherNodesStream extends TupleStream implements Expressible {
          params,
          metrics,
          trackTraversal,
-         scatter);
+         scatter,
+         docFreq);
   }
 
   private void init(String zkHost,
@@ -240,7 +246,8 @@ public class GatherNodesStream extends TupleStream implements Expressible {
                     Map queryParams,
                     List<Metric> metrics,
                     boolean trackTraversal,
-                    Set<Traversal.Scatter> scatter) {
+                    Set<Traversal.Scatter> scatter,
+                    int maxDocFreq) {
     this.zkHost = zkHost;
     this.collection = collection;
     this.tupleStream = tupleStream;
@@ -251,6 +258,7 @@ public class GatherNodesStream extends TupleStream implements Expressible {
     this.metrics = metrics;
     this.trackTraversal = trackTraversal;
     this.scatter = scatter;
+    this.maxDocFreq = maxDocFreq;
   }
 
   @Override
@@ -298,6 +306,9 @@ public class GatherNodesStream extends TupleStream implements Expressible {
 
     expression.addParameter(new StreamExpressionNamedParameter("zkHost", zkHost));
     expression.addParameter(new StreamExpressionNamedParameter("gather", zkHost));
+    if(maxDocFreq > -1) {
+      expression.addParameter(new StreamExpressionNamedParameter("maxDocFreq", Integer.toString(maxDocFreq)));
+    }
     expression.addParameter(new StreamExpressionNamedParameter("walk", traverseFrom+"->"+traverseTo));
     expression.addParameter(new StreamExpressionNamedParameter("trackTraversal", Boolean.toString(trackTraversal)));
 
@@ -410,7 +421,7 @@ public class GatherNodesStream extends TupleStream implements Expressible {
       }
 
       if(queryParams.containsKey("fl")) {
-        String flString = (String)queryParams.get("fl");
+        String flString = queryParams.get("fl");
         String[] flArray = flString.split(",");
         for(String f : flArray) {
           flSet.add(f.trim());
@@ -433,14 +444,22 @@ public class GatherNodesStream extends TupleStream implements Expressible {
 
       StringBuffer nodeQuery = new StringBuffer();
 
+      boolean comma = false;
       for(String node : nodes) {
-        nodeQuery.append(node).append(" ");
+        if(comma) {
+          nodeQuery.append(",");
+        }
+        nodeQuery.append(node);
+        comma = true;
       }
 
-      String q = traverseTo + ":(" + nodeQuery.toString().trim() + ")";
-
+      if(maxDocFreq > -1) {
+        String docFreqParam = " maxDocFreq="+maxDocFreq;
+        joinParams.put("q", "{!graphTerms f=" + traverseTo + docFreqParam + "}" + nodeQuery.toString());
+      } else {
+        joinParams.put("q", "{!terms f=" + traverseTo+"}" + nodeQuery.toString());
+      }
 
-      joinParams.put("q", q);
       TupleStream stream = null;
       try {
         stream = new UniqueStream(new CloudSolrStream(zkHost, collection, joinParams), new MultipleFieldEqualitor(new FieldEqualitor(gather), new FieldEqualitor(traverseTo)));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9ce830d8/solr/solrj/src/test/org/apache/solr/client/solrj/io/graph/GraphExpressionTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/graph/GraphExpressionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/graph/GraphExpressionTest.java
index b5231e2..7c1f97d 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/graph/GraphExpressionTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/graph/GraphExpressionTest.java
@@ -326,6 +326,23 @@ public class GraphExpressionTest extends AbstractFullDistribZkTestBase {
     assertTrue(tuples.get(2).getString("node").equals("basket3"));
     assertTrue(tuples.get(3).getString("node").equals("basket4"));
 
+
+    //Test maxDocFreq param
+    String docFreqExpr = "gatherNodes(collection1, " +
+                         "walk=\"product1, product7->product_s\"," +
+                         "maxDocFreq=\"2\","+
+                         "gather=\"basket_s\")";
+
+    stream = (GatherNodesStream)factory.constructStream(docFreqExpr);
+    stream.setStreamContext(context);
+
+    tuples = getTuples(stream);
+    Collections.sort(tuples, new FieldComparator("node", ComparatorOrder.ASCENDING));
+    assertTrue(tuples.size() == 1);
+    assertTrue(tuples.get(0).getString("node").equals("basket2"));
+
+
+
     String expr2 = "gatherNodes(collection1, " +
                                  expr+","+
                                 "walk=\"node->basket_s\"," +


[26/50] [abbrv] lucene-solr:solr-5750: LUCENE-7247: TestCoreParser.dumpResults verbose and test-fail logging tweaks

Posted by ds...@apache.org.
LUCENE-7247: TestCoreParser.dumpResults verbose and test-fail logging tweaks


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

Branch: refs/heads/solr-5750
Commit: fd7b2159d8cc5d814c4d7f47f64945d8f4f5426f
Parents: e3e9114
Author: Christine Poerschke <cp...@apache.org>
Authored: Mon Apr 25 09:39:29 2016 +0100
Committer: Christine Poerschke <cp...@apache.org>
Committed: Mon Apr 25 09:39:29 2016 +0100

----------------------------------------------------------------------
 .../org/apache/lucene/queryparser/xml/TestCoreParser.java   | 9 ++++++---
 1 file changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fd7b2159/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/TestCoreParser.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/TestCoreParser.java b/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/TestCoreParser.java
index 4cf5fcf..f252600 100644
--- a/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/TestCoreParser.java
+++ b/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/TestCoreParser.java
@@ -221,13 +221,15 @@ public class TestCoreParser extends LuceneTestCase {
 
   protected void dumpResults(String qType, Query q, int numDocs) throws IOException {
     if (VERBOSE) {
-      System.out.println("TEST: qType=" + qType + " query=" + q + " numDocs=" + numDocs);
+      System.out.println("TEST: qType=" + qType + " numDocs=" + numDocs + " " + q.getClass().getCanonicalName() + " query=" + q);
     }
     final IndexSearcher searcher = searcher();
     TopDocs hits = searcher.search(q, numDocs);
-    assertTrue(qType + " should produce results ", hits.totalHits > 0);
+    final boolean producedResults = (hits.totalHits > 0);
+    if (!producedResults) {
+      System.out.println("TEST: qType=" + qType + " numDocs=" + numDocs + " " + q.getClass().getCanonicalName() + " query=" + q);
+    }
     if (VERBOSE) {
-      System.out.println("=========" + qType + "============");
       ScoreDoc[] scoreDocs = hits.scoreDocs;
       for (int i = 0; i < Math.min(numDocs, hits.totalHits); i++) {
         Document ldoc = searcher.doc(scoreDocs[i].doc);
@@ -235,5 +237,6 @@ public class TestCoreParser extends LuceneTestCase {
       }
       System.out.println();
     }
+    assertTrue(qType + " produced no results", producedResults);
   }
 }


[17/50] [abbrv] lucene-solr:solr-5750: SOLR-9025: Add SolrCoreTest.testImplicitPlugins test.

Posted by ds...@apache.org.
SOLR-9025: Add SolrCoreTest.testImplicitPlugins test.


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

Branch: refs/heads/solr-5750
Commit: 666472b74f2063a2a894837ee3768335bcf7f36a
Parents: 776f9ec
Author: Christine Poerschke <cp...@apache.org>
Authored: Fri Apr 22 18:21:41 2016 +0100
Committer: Christine Poerschke <cp...@apache.org>
Committed: Fri Apr 22 18:21:41 2016 +0100

----------------------------------------------------------------------
 .../test/org/apache/solr/core/SolrCoreTest.java | 40 ++++++++++++++++++++
 1 file changed, 40 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/666472b7/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java b/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java
index 5482707..200935a 100644
--- a/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java
+++ b/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java
@@ -29,6 +29,7 @@ import org.apache.solr.util.plugin.SolrCoreAware;
 import org.junit.Test;
 
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.Callable;
@@ -69,6 +70,45 @@ public class SolrCoreTest extends SolrTestCaseJ4 {
   }
 
   @Test
+  public void testImplicitPlugins() {
+    final SolrCore core = h.getCore();
+    final List<PluginInfo> implicitHandlers = core.getImplicitHandlers();
+
+    final Map<String,String> pathToClassMap = new HashMap<>(implicitHandlers.size());
+    for (PluginInfo implicitHandler : implicitHandlers) {
+      assertEquals("wrong type for "+implicitHandler.toString(),
+          SolrRequestHandler.TYPE, implicitHandler.type);
+      pathToClassMap.put(implicitHandler.name, implicitHandler.className);
+    }
+
+    int ihCount = 0;
+    {
+      ++ihCount; assertEquals(pathToClassMap.get("/admin/file"), "solr.ShowFileRequestHandler");
+      ++ihCount; assertEquals(pathToClassMap.get("/admin/logging"), "solr.LoggingHandler");
+      ++ihCount; assertEquals(pathToClassMap.get("/admin/luke"), "solr.LukeRequestHandler");
+      ++ihCount; assertEquals(pathToClassMap.get("/admin/mbeans"), "solr.SolrInfoMBeanHandler");
+      ++ihCount; assertEquals(pathToClassMap.get("/admin/ping"), "solr.PingRequestHandler");
+      ++ihCount; assertEquals(pathToClassMap.get("/admin/plugins"), "solr.PluginInfoHandler");
+      ++ihCount; assertEquals(pathToClassMap.get("/admin/properties"), "solr.PropertiesRequestHandler");
+      ++ihCount; assertEquals(pathToClassMap.get("/admin/segments"), "solr.SegmentsInfoRequestHandler");
+      ++ihCount; assertEquals(pathToClassMap.get("/admin/system"), "solr.SystemInfoHandler");
+      ++ihCount; assertEquals(pathToClassMap.get("/admin/threads"), "solr.ThreadDumpHandler");
+      ++ihCount; assertEquals(pathToClassMap.get("/config"), "solr.SolrConfigHandler");
+      ++ihCount; assertEquals(pathToClassMap.get("/export"), "solr.SearchHandler");
+      ++ihCount; assertEquals(pathToClassMap.get("/get"), "solr.RealTimeGetHandler");
+      ++ihCount; assertEquals(pathToClassMap.get("/replication"), "solr.ReplicationHandler");
+      ++ihCount; assertEquals(pathToClassMap.get("/schema"), "solr.SchemaHandler");
+      ++ihCount; assertEquals(pathToClassMap.get("/sql"), "solr.SQLHandler");
+      ++ihCount; assertEquals(pathToClassMap.get("/stream"), "solr.StreamHandler");
+      ++ihCount; assertEquals(pathToClassMap.get("/update"), "solr.UpdateRequestHandler");
+      ++ihCount; assertEquals(pathToClassMap.get("/update/csv"), "solr.UpdateRequestHandler");
+      ++ihCount; assertEquals(pathToClassMap.get("/update/json"), "solr.UpdateRequestHandler");
+      ++ihCount; assertEquals(pathToClassMap.get("/update/json/docs"), "solr.UpdateRequestHandler");
+    }
+    assertEquals("wrong number of implicit handlers", ihCount, implicitHandlers.size());
+  }
+
+  @Test
   public void testClose() throws Exception {
     final CoreContainer cores = h.getCoreContainer();
     SolrCore core = cores.getCore(SolrTestCaseJ4.DEFAULT_TEST_CORENAME);


[29/50] [abbrv] lucene-solr:solr-5750: LUCENE-7251: remove LatLonGrid, remove slow polygon methods, speed up multiple components

Posted by ds...@apache.org.
LUCENE-7251: remove LatLonGrid, remove slow polygon methods, speed up multiple components


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

Branch: refs/heads/solr-5750
Commit: 837264a42ebe3e6091a64b2d0410ee7c63eebb1f
Parents: fe795c9
Author: Robert Muir <rm...@apache.org>
Authored: Mon Apr 25 15:29:54 2016 -0400
Committer: Robert Muir <rm...@apache.org>
Committed: Mon Apr 25 15:31:54 2016 -0400

----------------------------------------------------------------------
 .../src/java/org/apache/lucene/geo/Polygon.java | 221 ---------
 .../java/org/apache/lucene/geo/Polygon2D.java   | 473 +++++++++++++++++++
 .../test/org/apache/lucene/geo/TestPolygon.java | 330 -------------
 .../org/apache/lucene/geo/TestPolygon2D.java    | 289 +++++++++++
 .../org/apache/lucene/document/LatLonGrid.java  | 168 -------
 .../document/LatLonPointInPolygonQuery.java     |  22 +-
 .../org/apache/lucene/document/LatLonTree.java  | 396 ----------------
 .../apache/lucene/document/TestLatLonGrid.java  | 106 -----
 .../apache/lucene/document/TestLatLonTree.java  |  53 ---
 .../search/GeoPointInPolygonQueryImpl.java      |  16 +-
 .../apache/lucene/geo/BaseGeoPointTestCase.java |   2 +-
 .../java/org/apache/lucene/geo/GeoTestUtil.java |  54 +++
 12 files changed, 830 insertions(+), 1300 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/837264a4/lucene/core/src/java/org/apache/lucene/geo/Polygon.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/geo/Polygon.java b/lucene/core/src/java/org/apache/lucene/geo/Polygon.java
index 361c199..3b5dec9 100644
--- a/lucene/core/src/java/org/apache/lucene/geo/Polygon.java
+++ b/lucene/core/src/java/org/apache/lucene/geo/Polygon.java
@@ -18,8 +18,6 @@ package org.apache.lucene.geo;
 
 import java.util.Arrays;
 
-import org.apache.lucene.index.PointValues.Relation;
-
 /**
  * Represents a closed polygon on the earth's surface.
  * <p>
@@ -48,8 +46,6 @@ public final class Polygon {
   /** maximum longitude of this polygon's bounding box area */
   public final double maxLon;
 
-  // TODO: we could also compute the maximal inner bounding box, to make relations faster to compute?
-
   /**
    * Creates a new Polygon from the supplied latitude/longitude array, and optionally any holes.
    */
@@ -110,200 +106,6 @@ public final class Polygon {
     this.maxLon = maxLon;
   }
 
-  /** 
-   * Returns true if the point is contained within this polygon.
-   * <p>
-   * See <a href="https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html">
-   * https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html</a> for more information.
-   */
-  // ported to java from https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html
-  // original code under the BSD license (https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html#License%20to%20Use)
-  //
-  // Copyright (c) 1970-2003, Wm. Randolph Franklin
-  //
-  // Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated 
-  // documentation files (the "Software"), to deal in the Software without restriction, including without limitation 
-  // the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and 
-  // to permit persons to whom the Software is furnished to do so, subject to the following conditions:
-  //
-  // 1. Redistributions of source code must retain the above copyright 
-  //    notice, this list of conditions and the following disclaimers.
-  // 2. Redistributions in binary form must reproduce the above copyright 
-  //    notice in the documentation and/or other materials provided with 
-  //    the distribution.
-  // 3. The name of W. Randolph Franklin may not be used to endorse or 
-  //    promote products derived from this Software without specific 
-  //    prior written permission. 
-  //
-  // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED 
-  // TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL 
-  // THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF 
-  // CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS 
-  // IN THE SOFTWARE. 
-  public boolean contains(double latitude, double longitude) {
-    // check bounding box
-    if (latitude < minLat || latitude > maxLat || longitude < minLon || longitude > maxLon) {
-      return false;
-    }
-    
-    boolean inPoly = false;
-    boolean previous = polyLats[0] > latitude;
-    for (int i = 1; i < polyLats.length; i++) {
-      boolean current = polyLats[i] > latitude;
-      if (current != previous) {
-        if (longitude < (polyLons[i-1] - polyLons[i]) * (latitude - polyLats[i]) / (polyLats[i-1] - polyLats[i]) + polyLons[i]) {
-          inPoly = !inPoly;
-        }
-        previous = current;
-      }
-    }
-    if (inPoly) {
-      for (Polygon hole : holes) {
-        if (hole.contains(latitude, longitude)) {
-          return false;
-        }
-      }
-      return true;
-    } else {
-      return false;
-    }
-  }
-  
-  /** Returns relation to the provided rectangle */
-  public Relation relate(double minLat, double maxLat, double minLon, double maxLon) {
-    // if the bounding boxes are disjoint then the shape does not cross
-    if (maxLon < this.minLon || minLon > this.maxLon || maxLat < this.minLat || minLat > this.maxLat) {
-      return Relation.CELL_OUTSIDE_QUERY;
-    }
-    // if the rectangle fully encloses us, we cross.
-    if (minLat <= this.minLat && maxLat >= this.maxLat && minLon <= this.minLon && maxLon >= this.maxLon) {
-      return Relation.CELL_CROSSES_QUERY;
-    }
-    // check any holes
-    for (Polygon hole : holes) {
-      Relation holeRelation = hole.relate(minLat, maxLat, minLon, maxLon);
-      if (holeRelation == Relation.CELL_CROSSES_QUERY) {
-        return Relation.CELL_CROSSES_QUERY;
-      } else if (holeRelation == Relation.CELL_INSIDE_QUERY) {
-        return Relation.CELL_OUTSIDE_QUERY;
-      }
-    }
-    // check each corner: if < 4 are present, its cheaper than crossesSlowly
-    int numCorners = numberOfCorners(minLat, maxLat, minLon, maxLon);
-    if (numCorners == 4) {
-      if (crossesSlowly(minLat, maxLat, minLon, maxLon)) {
-        return Relation.CELL_CROSSES_QUERY;
-      }
-      return Relation.CELL_INSIDE_QUERY;
-    } else if (numCorners > 0) {
-      return Relation.CELL_CROSSES_QUERY;
-    }
-    
-    // we cross
-    if (crossesSlowly(minLat, maxLat, minLon, maxLon)) {
-      return Relation.CELL_CROSSES_QUERY;
-    }
-    
-    return Relation.CELL_OUTSIDE_QUERY;
-  }
-  
-  // returns 0, 4, or something in between
-  private int numberOfCorners(double minLat, double maxLat, double minLon, double maxLon) {
-    int containsCount = 0;
-    if (contains(minLat, minLon)) {
-      containsCount++;
-    }
-    if (contains(minLat, maxLon)) {
-      containsCount++;
-    }
-    if (containsCount == 1) {
-      return containsCount;
-    }
-    if (contains(maxLat, maxLon)) {
-      containsCount++;
-    }
-    if (containsCount == 2) {
-      return containsCount;
-    }
-    if (contains(maxLat, minLon)) {
-      containsCount++;
-    }
-    return containsCount;
-  }
-
-  /** Returns true if the box crosses our polygon */
-  private boolean crossesSlowly(double minLat, double maxLat, double minLon, double maxLon) {
-    // we compute line intersections of every polygon edge with every box line.
-    // if we find one, return true.
-    // for each box line (AB):
-    //   for each poly line (CD):
-    //     intersects = orient(C,D,A) * orient(C,D,B) <= 0 && orient(A,B,C) * orient(A,B,D) <= 0
-    for (int i = 1; i < polyLons.length; i++) {
-      double cy = polyLats[i - 1];
-      double dy = polyLats[i];
-      double cx = polyLons[i - 1];
-      double dx = polyLons[i];
-
-      // optimization: see if the rectangle is outside of the "bounding box" of the polyline at all
-      // if not, don't waste our time trying more complicated stuff
-      if ((cy < minLat && dy < minLat) ||
-          (cy > maxLat && dy > maxLat) ||
-          (cx < minLon && dx < minLon) ||
-          (cx > maxLon && dx > maxLon)) {
-        continue;
-      }
-
-      // does box's top edge intersect polyline?
-      // ax = minLon, bx = maxLon, ay = maxLat, by = maxLat
-      if (orient(cx, cy, dx, dy, minLon, maxLat) * orient(cx, cy, dx, dy, maxLon, maxLat) <= 0 &&
-          orient(minLon, maxLat, maxLon, maxLat, cx, cy) * orient(minLon, maxLat, maxLon, maxLat, dx, dy) <= 0) {
-        return true;
-      }
-
-      // does box's right edge intersect polyline?
-      // ax = maxLon, bx = maxLon, ay = maxLat, by = minLat
-      if (orient(cx, cy, dx, dy, maxLon, maxLat) * orient(cx, cy, dx, dy, maxLon, minLat) <= 0 &&
-          orient(maxLon, maxLat, maxLon, minLat, cx, cy) * orient(maxLon, maxLat, maxLon, minLat, dx, dy) <= 0) {
-        return true;
-      }
-
-      // does box's bottom edge intersect polyline?
-      // ax = maxLon, bx = minLon, ay = minLat, by = minLat
-      if (orient(cx, cy, dx, dy, maxLon, minLat) * orient(cx, cy, dx, dy, minLon, minLat) <= 0 &&
-          orient(maxLon, minLat, minLon, minLat, cx, cy) * orient(maxLon, minLat, minLon, minLat, dx, dy) <= 0) {
-        return true;
-      }
-
-      // does box's left edge intersect polyline?
-      // ax = minLon, bx = minLon, ay = minLat, by = maxLat
-      if (orient(cx, cy, dx, dy, minLon, minLat) * orient(cx, cy, dx, dy, minLon, maxLat) <= 0 &&
-          orient(minLon, minLat, minLon, maxLat, cx, cy) * orient(minLon, minLat, minLon, maxLat, dx, dy) <= 0) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  /**
-   * Returns a positive value if points a, b, and c are arranged in counter-clockwise order,
-   * negative value if clockwise, zero if collinear.
-   */
-  // see the "Orient2D" method described here:
-  // http://www.cs.berkeley.edu/~jrs/meshpapers/robnotes.pdf
-  // https://www.cs.cmu.edu/~quake/robust.html
-  // Note that this one does not yet have the floating point tricks to be exact!
-  private static int orient(double ax, double ay, double bx, double by, double cx, double cy) {
-    double v1 = (bx - ax) * (cy - ay);
-    double v2 = (cx - ax) * (by - ay);
-    if (v1 > v2) {
-      return 1;
-    } else if (v1 < v2) {
-      return -1;
-    } else {
-      return 0;
-    }
-  }
-
   /** Returns a copy of the internal latitude array */
   public double[] getPolyLats() {
     return polyLats.clone();
@@ -319,29 +121,6 @@ public final class Polygon {
     return holes.clone();
   }
 
-  /** Helper for multipolygon logic: returns true if any of the supplied polygons contain the point */
-  public static boolean contains(Polygon[] polygons, double latitude, double longitude) {
-    for (Polygon polygon : polygons) {
-      if (polygon.contains(latitude, longitude)) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  /** Returns the multipolygon relation for the rectangle */
-  public static Relation relate(Polygon[] polygons, double minLat, double maxLat, double minLon, double maxLon) {
-    for (Polygon polygon : polygons) {
-      Relation relation = polygon.relate(minLat, maxLat, minLon, maxLon);
-      if (relation != Relation.CELL_OUTSIDE_QUERY) {
-        // note: we optimize for non-overlapping multipolygons. so if we cross one,
-        // we won't keep iterating to try to find a contains.
-        return relation;
-      }
-    }
-    return Relation.CELL_OUTSIDE_QUERY;
-  }
-
   @Override
   public int hashCode() {
     final int prime = 31;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/837264a4/lucene/core/src/java/org/apache/lucene/geo/Polygon2D.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/geo/Polygon2D.java b/lucene/core/src/java/org/apache/lucene/geo/Polygon2D.java
new file mode 100644
index 0000000..320d71a
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/geo/Polygon2D.java
@@ -0,0 +1,473 @@
+/*
+ * 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.geo;
+
+import java.util.Arrays;
+
+import org.apache.lucene.geo.Polygon;
+import org.apache.lucene.index.PointValues.Relation;
+
+/**
+ * 2D polygon implementation represented as a balanced interval tree of edges.
+ * <p>
+ * Construction takes {@code O(n log n)} time for sorting and tree construction.
+ * {@link #contains contains()} and {@link #relate relate()} are {@code O(n)}, but for most 
+ * practical polygons are much faster than brute force.
+ * <p>
+ * Loosely based on the algorithm described in <a href="http://www-ma2.upc.es/geoc/Schirra-pointPolygon.pdf">
+ * http://www-ma2.upc.es/geoc/Schirra-pointPolygon.pdf</a>.
+ * @lucene.internal
+ */
+// Both Polygon.contains() and Polygon.crossesSlowly() loop all edges, and first check that the edge is within a range.
+// we just organize the edges to do the same computations on the same subset of edges more efficiently. 
+public final class Polygon2D {
+  /** minimum latitude of this polygon's bounding box area */
+  public final double minLat;
+  /** maximum latitude of this polygon's bounding box area */
+  public final double maxLat;
+  /** minimum longitude of this polygon's bounding box area */
+  public final double minLon;
+  /** maximum longitude of this polygon's bounding box area */
+  public final double maxLon;
+  
+  // each component/hole is a node in an augmented 2d kd-tree: we alternate splitting between latitude/longitude,
+  // and pull up max values for both dimensions to each parent node (regardless of split).
+
+  /** maximum latitude of this component or any of its children */
+  private double maxY;
+  /** maximum longitude of this component or any of its children */
+  private double maxX;
+  /** which dimension was this node split on */
+  // TODO: its implicit based on level, but boolean keeps code simple
+  private boolean splitX;
+
+  // child components, or null
+  private Polygon2D left;
+  private Polygon2D right;
+  
+  /** tree of holes, or null */
+  private final Polygon2D holes;
+  
+  /** root node of edge tree */
+  private final Edge tree;
+
+  private Polygon2D(Polygon polygon, Polygon2D holes) {
+    this.holes = holes;
+    this.minLat = polygon.minLat;
+    this.maxLat = polygon.maxLat;
+    this.minLon = polygon.minLon;
+    this.maxLon = polygon.maxLon;
+    this.maxY = maxLat;
+    this.maxX = maxLon;
+    
+    // create interval tree of edges
+    this.tree = createTree(polygon.getPolyLats(), polygon.getPolyLons());
+  }
+
+  /** 
+   * Returns true if the point is contained within this polygon.
+   * <p>
+   * See <a href="https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html">
+   * https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html</a> for more information.
+   */
+  public boolean contains(double latitude, double longitude) {
+    if (latitude <= maxY && longitude <= maxX) {
+      if (componentContains(latitude, longitude)) {
+        return true;
+      }
+      if (left != null) {
+        if (left.contains(latitude, longitude)) {
+          return true;
+        }
+      }
+      if (right != null && ((splitX == false && latitude >= minLat) || (splitX && longitude >= minLon))) {
+        if (right.contains(latitude, longitude)) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+  
+  /** Returns true if the point is contained within this polygon component. */
+  private boolean componentContains(double latitude, double longitude) {
+    // check bounding box
+    if (latitude < minLat || latitude > maxLat || longitude < minLon || longitude > maxLon) {
+      return false;
+    }
+    
+    if (tree.contains(latitude, longitude)) {
+      if (holes != null && holes.contains(latitude, longitude)) {
+        return false;
+      }
+      return true;
+    }
+    
+    return false;
+  }
+  
+  /** Returns relation to the provided rectangle */
+  public Relation relate(double minLat, double maxLat, double minLon, double maxLon) {
+    if (minLat <= maxY && minLon <= maxX) {
+      Relation relation = componentRelate(minLat, maxLat, minLon, maxLon);
+      if (relation != Relation.CELL_OUTSIDE_QUERY) {
+        return relation;
+      }
+      if (left != null) {
+        relation = left.relate(minLat, maxLat, minLon, maxLon);
+        if (relation != Relation.CELL_OUTSIDE_QUERY) {
+          return relation;
+        }
+      }
+      if (right != null && ((splitX == false && maxLat >= this.minLat) || (splitX && maxLon >= this.minLon))) {
+        relation = right.relate(minLat, maxLat, minLon, maxLon);
+        if (relation != Relation.CELL_OUTSIDE_QUERY) {
+          return relation;
+        }
+      }
+    }
+    return Relation.CELL_OUTSIDE_QUERY;
+  }
+
+  /** Returns relation to the provided rectangle for this component */
+  private Relation componentRelate(double minLat, double maxLat, double minLon, double maxLon) {
+    // if the bounding boxes are disjoint then the shape does not cross
+    if (maxLon < this.minLon || minLon > this.maxLon || maxLat < this.minLat || minLat > this.maxLat) {
+      return Relation.CELL_OUTSIDE_QUERY;
+    }
+    // if the rectangle fully encloses us, we cross.
+    if (minLat <= this.minLat && maxLat >= this.maxLat && minLon <= this.minLon && maxLon >= this.maxLon) {
+      return Relation.CELL_CROSSES_QUERY;
+    }
+    // check any holes
+    if (holes != null) {
+      Relation holeRelation = holes.relate(minLat, maxLat, minLon, maxLon);
+      if (holeRelation == Relation.CELL_CROSSES_QUERY) {
+        return Relation.CELL_CROSSES_QUERY;
+      } else if (holeRelation == Relation.CELL_INSIDE_QUERY) {
+        return Relation.CELL_OUTSIDE_QUERY;
+      }
+    }
+    // check each corner: if < 4 are present, its cheaper than crossesSlowly
+    int numCorners = numberOfCorners(minLat, maxLat, minLon, maxLon);
+    if (numCorners == 4) {
+      if (tree.crosses(minLat, maxLat, minLon, maxLon)) {
+        return Relation.CELL_CROSSES_QUERY;
+      }
+      return Relation.CELL_INSIDE_QUERY;
+    } else if (numCorners > 0) {
+      return Relation.CELL_CROSSES_QUERY;
+    }
+    
+    // we cross
+    if (tree.crosses(minLat, maxLat, minLon, maxLon)) {
+      return Relation.CELL_CROSSES_QUERY;
+    }
+    
+    return Relation.CELL_OUTSIDE_QUERY;
+  }
+  
+  // returns 0, 4, or something in between
+  private int numberOfCorners(double minLat, double maxLat, double minLon, double maxLon) {
+    int containsCount = 0;
+    if (componentContains(minLat, minLon)) {
+      containsCount++;
+    }
+    if (componentContains(minLat, maxLon)) {
+      containsCount++;
+    }
+    if (containsCount == 1) {
+      return containsCount;
+    }
+    if (componentContains(maxLat, maxLon)) {
+      containsCount++;
+    }
+    if (containsCount == 2) {
+      return containsCount;
+    }
+    if (componentContains(maxLat, minLon)) {
+      containsCount++;
+    }
+    return containsCount;
+  }
+  
+  /** Creates tree from sorted components (with range low and high inclusive) */
+  private static Polygon2D createTree(Polygon2D components[], int low, int high, boolean splitX) {
+    if (low > high) {
+      return null;
+    } else if (low < high) {
+      // TODO: do one sort instead! there are better algorithms!
+      if (splitX) {
+        Arrays.sort(components, low, high+1, (left, right) -> {
+          int ret = Double.compare(left.minLon, right.minLon);
+          if (ret == 0) {
+            ret = Double.compare(left.maxX, right.maxX);
+          }
+          return ret;
+        });
+      } else {
+        Arrays.sort(components, low, high+1, (left, right) -> {
+          int ret = Double.compare(left.minLat, right.minLat);
+          if (ret == 0) {
+            ret = Double.compare(left.maxY, right.maxY);
+          }
+          return ret;
+        });
+      }
+    }
+    // add midpoint
+    int mid = (low + high) >>> 1;
+    Polygon2D newNode = components[mid];
+    newNode.splitX = splitX;
+    // add children
+    newNode.left = createTree(components, low, mid - 1, !splitX);
+    newNode.right = createTree(components, mid + 1, high, !splitX);
+    // pull up max values to this node
+    if (newNode.left != null) {
+      newNode.maxX = Math.max(newNode.maxX, newNode.left.maxX);
+      newNode.maxY = Math.max(newNode.maxY, newNode.left.maxY);
+    }
+    if (newNode.right != null) {
+      newNode.maxX = Math.max(newNode.maxX, newNode.right.maxX);
+      newNode.maxY = Math.max(newNode.maxY, newNode.right.maxY);
+    }
+    return newNode;
+  }
+  
+  /** Builds a Polygon2D from multipolygon */
+  public static Polygon2D create(Polygon... polygons) {
+    Polygon2D components[] = new Polygon2D[polygons.length];
+    for (int i = 0; i < components.length; i++) {
+      Polygon gon = polygons[i];
+      Polygon gonHoles[] = gon.getHoles();
+      Polygon2D holes = null;
+      if (gonHoles.length > 0) {
+        holes = create(gonHoles);
+      }
+      components[i] = new Polygon2D(gon, holes);
+    }
+    return createTree(components, 0, components.length - 1, false);
+  }
+
+  /** 
+   * Internal tree node: represents polygon edge from lat1,lon1 to lat2,lon2.
+   * The sort value is {@code low}, which is the minimum latitude of the edge.
+   * {@code max} stores the maximum latitude of this edge or any children.
+   */
+  static final class Edge {
+    // lat-lon pair (in original order) of the two vertices
+    final double lat1, lat2;
+    final double lon1, lon2;
+    /** min of this edge */
+    final double low;
+    /** max latitude of this edge or any children */
+    double max;
+    
+    /** left child edge, or null */
+    Edge left;
+    /** right child edge, or null */
+    Edge right;
+
+    Edge(double lat1, double lon1, double lat2, double lon2, double low, double max) {
+      this.lat1 = lat1;
+      this.lon1 = lon1;
+      this.lat2 = lat2;
+      this.lon2 = lon2;
+      this.low = low;
+      this.max = max;
+    }
+    
+    /** 
+     * Returns true if the point crosses this edge subtree an odd number of times
+     * <p>
+     * See <a href="https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html">
+     * https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html</a> for more information.
+     */
+    // ported to java from https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html
+    // original code under the BSD license (https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html#License%20to%20Use)
+    //
+    // Copyright (c) 1970-2003, Wm. Randolph Franklin
+    //
+    // Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated 
+    // documentation files (the "Software"), to deal in the Software without restriction, including without limitation 
+    // the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and 
+    // to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+    //
+    // 1. Redistributions of source code must retain the above copyright 
+    //    notice, this list of conditions and the following disclaimers.
+    // 2. Redistributions in binary form must reproduce the above copyright 
+    //    notice in the documentation and/or other materials provided with 
+    //    the distribution.
+    // 3. The name of W. Randolph Franklin may not be used to endorse or 
+    //    promote products derived from this Software without specific 
+    //    prior written permission. 
+    //
+    // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED 
+    // TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL 
+    // THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF 
+    // CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS 
+    // IN THE SOFTWARE. 
+    boolean contains(double latitude, double longitude) {
+      // crossings algorithm is an odd-even algorithm, so we descend the tree xor'ing results along our path
+      boolean res = false;
+      if (latitude <= max) {
+        if (lat1 > latitude != lat2 > latitude) {
+          if (longitude < (lon1 - lon2) * (latitude - lat2) / (lat1 - lat2) + lon2) {
+            res = true;
+          }
+        }
+        if (left != null) {
+          res ^= left.contains(latitude, longitude);
+        }
+        if (right != null && latitude >= low) {
+          res ^= right.contains(latitude, longitude);
+        }
+      }
+      return res;
+    }
+    
+    /** Returns true if the box crosses any edge in this edge subtree */
+    boolean crosses(double minLat, double maxLat, double minLon, double maxLon) {
+      // we just have to cross one edge to answer the question, so we descend the tree and return when we do.
+      if (minLat <= max) {
+        // we compute line intersections of every polygon edge with every box line.
+        // if we find one, return true.
+        // for each box line (AB):
+        //   for each poly line (CD):
+        //     intersects = orient(C,D,A) * orient(C,D,B) <= 0 && orient(A,B,C) * orient(A,B,D) <= 0
+        double cy = lat1;
+        double dy = lat2;
+        double cx = lon1;
+        double dx = lon2;
+        
+        // optimization: see if the rectangle is outside of the "bounding box" of the polyline at all
+        // if not, don't waste our time trying more complicated stuff
+        boolean outside = (cy < minLat && dy < minLat) ||
+                          (cy > maxLat && dy > maxLat) ||
+                          (cx < minLon && dx < minLon) ||
+                          (cx > maxLon && dx > maxLon);
+        if (outside == false) {
+          // does box's top edge intersect polyline?
+          // ax = minLon, bx = maxLon, ay = maxLat, by = maxLat
+          if (orient(cx, cy, dx, dy, minLon, maxLat) * orient(cx, cy, dx, dy, maxLon, maxLat) <= 0 &&
+              orient(minLon, maxLat, maxLon, maxLat, cx, cy) * orient(minLon, maxLat, maxLon, maxLat, dx, dy) <= 0) {
+            return true;
+          }
+
+          // does box's right edge intersect polyline?
+          // ax = maxLon, bx = maxLon, ay = maxLat, by = minLat
+          if (orient(cx, cy, dx, dy, maxLon, maxLat) * orient(cx, cy, dx, dy, maxLon, minLat) <= 0 &&
+              orient(maxLon, maxLat, maxLon, minLat, cx, cy) * orient(maxLon, maxLat, maxLon, minLat, dx, dy) <= 0) {
+            return true;
+          }
+
+          // does box's bottom edge intersect polyline?
+          // ax = maxLon, bx = minLon, ay = minLat, by = minLat
+          if (orient(cx, cy, dx, dy, maxLon, minLat) * orient(cx, cy, dx, dy, minLon, minLat) <= 0 &&
+              orient(maxLon, minLat, minLon, minLat, cx, cy) * orient(maxLon, minLat, minLon, minLat, dx, dy) <= 0) {
+            return true;
+          }
+
+          // does box's left edge intersect polyline?
+          // ax = minLon, bx = minLon, ay = minLat, by = maxLat
+          if (orient(cx, cy, dx, dy, minLon, minLat) * orient(cx, cy, dx, dy, minLon, maxLat) <= 0 &&
+              orient(minLon, minLat, minLon, maxLat, cx, cy) * orient(minLon, minLat, minLon, maxLat, dx, dy) <= 0) {
+            return true;
+          }
+        }
+        
+        if (left != null) {
+          if (left.crosses(minLat, maxLat, minLon, maxLon)) {
+            return true;
+          }
+        }
+        
+        if (right != null && maxLat >= low) {
+          if (right.crosses(minLat, maxLat, minLon, maxLon)) {
+            return true;
+          }
+        }
+      }
+      return false;
+    }
+  }
+
+  /** 
+   * Creates an edge interval tree from a set of polygon vertices.
+   * @return root node of the tree.
+   */
+  private static Edge createTree(double polyLats[], double polyLons[]) {
+    Edge edges[] = new Edge[polyLats.length - 1];
+    for (int i = 1; i < polyLats.length; i++) {
+      double lat1 = polyLats[i-1];
+      double lon1 = polyLons[i-1];
+      double lat2 = polyLats[i];
+      double lon2 = polyLons[i];
+      edges[i - 1] = new Edge(lat1, lon1, lat2, lon2, Math.min(lat1, lat2), Math.max(lat1, lat2));
+    }
+    // sort the edges then build a balanced tree from them
+    Arrays.sort(edges, (left, right) -> {
+      int ret = Double.compare(left.low, right.low);
+      if (ret == 0) {
+        ret = Double.compare(left.max, right.max);
+      }
+      return ret;
+    });
+    return createTree(edges, 0, edges.length - 1);
+  }
+
+  /** Creates tree from sorted edges (with range low and high inclusive) */
+  private static Edge createTree(Edge edges[], int low, int high) {
+    if (low > high) {
+      return null;
+    }
+    // add midpoint
+    int mid = (low + high) >>> 1;
+    Edge newNode = edges[mid];
+    // add children
+    newNode.left = createTree(edges, low, mid - 1);
+    newNode.right = createTree(edges, mid + 1, high);
+    // pull up max values to this node
+    if (newNode.left != null) {
+      newNode.max = Math.max(newNode.max, newNode.left.max);
+    }
+    if (newNode.right != null) {
+      newNode.max = Math.max(newNode.max, newNode.right.max);
+    }
+    return newNode;
+  }
+
+  /**
+   * Returns a positive value if points a, b, and c are arranged in counter-clockwise order,
+   * negative value if clockwise, zero if collinear.
+   */
+  // see the "Orient2D" method described here:
+  // http://www.cs.berkeley.edu/~jrs/meshpapers/robnotes.pdf
+  // https://www.cs.cmu.edu/~quake/robust.html
+  // Note that this one does not yet have the floating point tricks to be exact!
+  private static int orient(double ax, double ay, double bx, double by, double cx, double cy) {
+    double v1 = (bx - ax) * (cy - ay);
+    double v2 = (cx - ax) * (by - ay);
+    if (v1 > v2) {
+      return 1;
+    } else if (v1 < v2) {
+      return -1;
+    } else {
+      return 0;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/837264a4/lucene/core/src/test/org/apache/lucene/geo/TestPolygon.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/geo/TestPolygon.java b/lucene/core/src/test/org/apache/lucene/geo/TestPolygon.java
index 12c3690..401092f 100644
--- a/lucene/core/src/test/org/apache/lucene/geo/TestPolygon.java
+++ b/lucene/core/src/test/org/apache/lucene/geo/TestPolygon.java
@@ -16,17 +16,8 @@
  */
 package org.apache.lucene.geo;
 
-import org.apache.lucene.geo.Polygon;
-import org.apache.lucene.index.PointValues.Relation;
 import org.apache.lucene.util.LuceneTestCase;
 
-import static org.apache.lucene.geo.GeoTestUtil.nextLatitude;
-import static org.apache.lucene.geo.GeoTestUtil.nextLongitude;
-import static org.apache.lucene.geo.GeoTestUtil.nextPolygon;
-
-import java.util.ArrayList;
-import java.util.List;
-
 public class TestPolygon extends LuceneTestCase {
   
   /** null polyLats not allowed */
@@ -68,325 +59,4 @@ public class TestPolygon extends LuceneTestCase {
     });
     assertTrue(expected.getMessage(), expected.getMessage().contains("it must close itself"));
   }
-  
-  /** Three boxes, an island inside a hole inside a shape */
-  public void testMultiPolygon() {
-    Polygon hole = new Polygon(new double[] { -10, -10, 10, 10, -10 }, new double[] { -10, 10, 10, -10, -10 });
-    Polygon outer = new Polygon(new double[] { -50, -50, 50, 50, -50 }, new double[] { -50, 50, 50, -50, -50 }, hole);
-    Polygon island = new Polygon(new double[] { -5, -5, 5, 5, -5 }, new double[] { -5, 5, 5, -5, -5 } );
-    Polygon polygons[] = new Polygon[] { outer, island };
-    
-    // contains(point)
-    assertTrue(Polygon.contains(polygons, -2, 2)); // on the island
-    assertFalse(Polygon.contains(polygons, -6, 6)); // in the hole
-    assertTrue(Polygon.contains(polygons, -25, 25)); // on the mainland
-    assertFalse(Polygon.contains(polygons, -51, 51)); // in the ocean
-    
-    // relate(box): this can conservatively return CELL_CROSSES_QUERY
-    assertEquals(Relation.CELL_INSIDE_QUERY, Polygon.relate(polygons, -2, 2, -2, 2)); // on the island
-    assertEquals(Relation.CELL_OUTSIDE_QUERY, Polygon.relate(polygons, 6, 7, 6, 7)); // in the hole
-    assertEquals(Relation.CELL_INSIDE_QUERY, Polygon.relate(polygons, 24, 25, 24, 25)); // on the mainland
-    assertEquals(Relation.CELL_OUTSIDE_QUERY, Polygon.relate(polygons, 51, 52, 51, 52)); // in the ocean
-    assertEquals(Relation.CELL_CROSSES_QUERY, Polygon.relate(polygons, -60, 60, -60, 60)); // enclosing us completely
-    assertEquals(Relation.CELL_CROSSES_QUERY, Polygon.relate(polygons, 49, 51, 49, 51)); // overlapping the mainland
-    assertEquals(Relation.CELL_CROSSES_QUERY, Polygon.relate(polygons, 9, 11, 9, 11)); // overlapping the hole
-    assertEquals(Relation.CELL_CROSSES_QUERY, Polygon.relate(polygons, 5, 6, 5, 6)); // overlapping the island
-  }
-  
-  public void testPacMan() throws Exception {
-    // pacman
-    double[] px = {0, 10, 10, 0, -8, -10, -8, 0, 10, 10, 0};
-    double[] py = {0, 5, 9, 10, 9, 0, -9, -10, -9, -5, 0};
-
-    // candidate crosses cell
-    double xMin = 2;//-5;
-    double xMax = 11;//0.000001;
-    double yMin = -1;//0;
-    double yMax = 1;//5;
-
-    // test cell crossing poly
-    Polygon polygon = new Polygon(py, px);
-    assertEquals(Relation.CELL_CROSSES_QUERY, polygon.relate(yMin, yMax, xMin, xMax));
-  }
-  
-  public void testBoundingBox() throws Exception {
-    for (int i = 0; i < 100; i++) {
-      Polygon polygon = nextPolygon();
-      
-      for (int j = 0; j < 100; j++) {
-        double latitude = nextLatitude();
-        double longitude = nextLongitude();
-        // if the point is within poly, then it should be in our bounding box
-        if (polygon.contains(latitude, longitude)) {
-          assertTrue(latitude >= polygon.minLat && latitude <= polygon.maxLat);
-          assertTrue(longitude >= polygon.minLon && longitude <= polygon.maxLon);
-        }
-      }
-    }
-  }
-  
-  // targets the bounding box directly
-  public void testBoundingBoxEdgeCases() throws Exception {
-    for (int i = 0; i < 100; i++) {
-      Polygon polygon = nextPolygon();
-      
-      for (int j = 0; j < 100; j++) {
-        double point[] = GeoTestUtil.nextPointNear(polygon);
-        double latitude = point[0];
-        double longitude = point[1];
-        // if the point is within poly, then it should be in our bounding box
-        if (polygon.contains(latitude, longitude)) {
-          assertTrue(latitude >= polygon.minLat && latitude <= polygon.maxLat);
-          assertTrue(longitude >= polygon.minLon && longitude <= polygon.maxLon);
-        }
-      }
-    }
-  }
-  
-  /** If polygon.contains(box) returns true, then any point in that box should return true as well */
-  public void testContainsRandom() throws Exception {
-    for (int i = 0; i < 1000; i++) {
-      Polygon polygon = nextPolygon();
-      
-      for (int j = 0; j < 100; j++) {
-        Rectangle rectangle = GeoTestUtil.nextBoxNear(polygon);
-        // allowed to conservatively return false
-        if (polygon.relate(rectangle.minLat, rectangle.maxLat, rectangle.minLon, rectangle.maxLon) == Relation.CELL_INSIDE_QUERY) {
-          for (int k = 0; k < 500; k++) {
-            // this tests in our range but sometimes outside! so we have to double-check its really in other box
-            double point[] = GeoTestUtil.nextPointNear(rectangle);
-            double latitude = point[0];
-            double longitude = point[1];
-            // check for sure its in our box
-            if (latitude >= rectangle.minLat && latitude <= rectangle.maxLat && longitude >= rectangle.minLon && longitude <= rectangle.maxLon) {
-              assertTrue(polygon.contains(latitude, longitude));
-            }
-          }
-          for (int k = 0; k < 100; k++) {
-            // this tests in our range but sometimes outside! so we have to double-check its really in other box
-            double point[] = GeoTestUtil.nextPointNear(polygon);
-            double latitude = point[0];
-            double longitude = point[1];
-            // check for sure its in our box
-            if (latitude >= rectangle.minLat && latitude <= rectangle.maxLat && longitude >= rectangle.minLon && longitude <= rectangle.maxLon) {
-              assertTrue(polygon.contains(latitude, longitude));
-            }
-          }
-        }
-      }
-    }
-  }
-  
-  /** If polygon.contains(box) returns true, then any point in that box should return true as well */
-  // different from testContainsRandom in that its not a purely random test. we iterate the vertices of the polygon
-  // and generate boxes near each one of those to try to be more efficient.
-  public void testContainsEdgeCases() throws Exception {
-    for (int i = 0; i < 1000; i++) {
-      Polygon polygon = nextPolygon();
-
-      for (int j = 0; j < 10; j++) {
-        Rectangle rectangle = GeoTestUtil.nextBoxNear(polygon);
-        // allowed to conservatively return false
-        if (polygon.relate(rectangle.minLat, rectangle.maxLat, rectangle.minLon, rectangle.maxLon) == Relation.CELL_INSIDE_QUERY) {
-          for (int k = 0; k < 100; k++) {
-            // this tests in our range but sometimes outside! so we have to double-check its really in other box
-            double point[] = GeoTestUtil.nextPointNear(rectangle);
-            double latitude = point[0];
-            double longitude = point[1];
-            // check for sure its in our box
-            if (latitude >= rectangle.minLat && latitude <= rectangle.maxLat && longitude >= rectangle.minLon && longitude <= rectangle.maxLon) {
-              assertTrue(polygon.contains(latitude, longitude));
-            }
-          }
-          for (int k = 0; k < 20; k++) {
-            // this tests in our range but sometimes outside! so we have to double-check its really in other box
-            double point[] = GeoTestUtil.nextPointNear(polygon);
-            double latitude = point[0];
-            double longitude = point[1];
-            // check for sure its in our box
-            if (latitude >= rectangle.minLat && latitude <= rectangle.maxLat && longitude >= rectangle.minLon && longitude <= rectangle.maxLon) {
-              assertTrue(polygon.contains(latitude, longitude));
-            }
-          }
-        }
-      }
-    }
-  }
-  
-  /** If polygon.intersects(box) returns false, then any point in that box should return false as well */
-  public void testIntersectRandom() {
-    for (int i = 0; i < 100; i++) {
-      Polygon polygon = nextPolygon();
-      
-      for (int j = 0; j < 100; j++) {
-        Rectangle rectangle = GeoTestUtil.nextBoxNear(polygon);
-        // allowed to conservatively return true.
-        if (polygon.relate(rectangle.minLat, rectangle.maxLat, rectangle.minLon, rectangle.maxLon) == Relation.CELL_OUTSIDE_QUERY) {
-          for (int k = 0; k < 1000; k++) {
-            double point[] = GeoTestUtil.nextPointNear(rectangle);
-            // this tests in our range but sometimes outside! so we have to double-check its really in other box
-            double latitude = point[0];
-            double longitude = point[1];
-            // check for sure its in our box
-            if (latitude >= rectangle.minLat && latitude <= rectangle.maxLat && longitude >= rectangle.minLon && longitude <= rectangle.maxLon) {
-              assertFalse(polygon.contains(latitude, longitude));
-            }
-          }
-          for (int k = 0; k < 100; k++) {
-            double point[] = GeoTestUtil.nextPointNear(polygon);
-            // this tests in our range but sometimes outside! so we have to double-check its really in other box
-            double latitude = point[0];
-            double longitude = point[1];
-            // check for sure its in our box
-            if (latitude >= rectangle.minLat && latitude <= rectangle.maxLat && longitude >= rectangle.minLon && longitude <= rectangle.maxLon) {
-              assertFalse(polygon.contains(latitude, longitude));
-            }
-          }
-        }
-      }
-    }
-  }
-  
-  /** If polygon.intersects(box) returns false, then any point in that box should return false as well */
-  // different from testIntersectsRandom in that its not a purely random test. we iterate the vertices of the polygon
-  // and generate boxes near each one of those to try to be more efficient.
-  public void testIntersectEdgeCases() {
-    for (int i = 0; i < 100; i++) {
-      Polygon polygon = nextPolygon();
-
-      for (int j = 0; j < 10; j++) {
-        Rectangle rectangle = GeoTestUtil.nextBoxNear(polygon);
-        // allowed to conservatively return false.
-        if (polygon.relate(rectangle.minLat, rectangle.maxLat, rectangle.minLon, rectangle.maxLon) == Relation.CELL_OUTSIDE_QUERY) {
-          for (int k = 0; k < 100; k++) {
-            // this tests in our range but sometimes outside! so we have to double-check its really in other box
-            double point[] = GeoTestUtil.nextPointNear(rectangle);
-            double latitude = point[0];
-            double longitude = point[1];
-            // check for sure its in our box
-            if (latitude >= rectangle.minLat && latitude <= rectangle.maxLat && longitude >= rectangle.minLon && longitude <= rectangle.maxLon) {
-              assertFalse(polygon.contains(latitude, longitude));
-            }
-          }
-          for (int k = 0; k < 50; k++) {
-            // this tests in our range but sometimes outside! so we have to double-check its really in other box
-            double point[] = GeoTestUtil.nextPointNear(polygon);
-            double latitude = point[0];
-            double longitude = point[1];
-            // check for sure its in our box
-            if (latitude >= rectangle.minLat && latitude <= rectangle.maxLat && longitude >= rectangle.minLon && longitude <= rectangle.maxLon) {
-              assertFalse(polygon.contains(latitude, longitude));
-            }
-          }
-        }
-      }
-    }
-  }
-  
-  /** Tests edge case behavior with respect to insideness */
-  public void testEdgeInsideness() {
-    Polygon poly = new Polygon(new double[] { -2, -2, 2, 2, -2 }, new double[] { -2, 2, 2, -2, -2 });
-    assertTrue(poly.contains(-2, -2)); // bottom left corner: true
-    assertFalse(poly.contains(-2, 2));  // bottom right corner: false
-    assertFalse(poly.contains(2, -2));  // top left corner: false
-    assertFalse(poly.contains(2,  2));  // top right corner: false
-    assertTrue(poly.contains(-2, -1)); // bottom side: true
-    assertTrue(poly.contains(-2, 0));  // bottom side: true
-    assertTrue(poly.contains(-2, 1));  // bottom side: true
-    assertFalse(poly.contains(2, -1));  // top side: false
-    assertFalse(poly.contains(2, 0));   // top side: false
-    assertFalse(poly.contains(2, 1));   // top side: false
-    assertFalse(poly.contains(-1, 2));  // right side: false
-    assertFalse(poly.contains(0, 2));   // right side: false
-    assertFalse(poly.contains(1, 2));   // right side: false
-    assertTrue(poly.contains(-1, -2)); // left side: true
-    assertTrue(poly.contains(0, -2));  // left side: true
-    assertTrue(poly.contains(1, -2));  // left side: true
-  }
-  
-  /** Tests that our impl supports multiple components and holes (not currently used) */
-  public void testMultiPolygonContains() {
-    // this is the equivalent of the following: we don't recommend anyone do this (e.g. relation logic will not work)
-    // but lets not lose the property that it works.
-    ///
-    // Polygon hole = new Polygon(new double[] { -10, -10, 10, 10, -10 }, new double[] { -10, 10, 10, -10, -10 });
-    // Polygon outer = new Polygon(new double[] { -50, -50, 50, 50, -50 }, new double[] { -50, 50, 50, -50, -50 }, hole);
-    // Polygon island = new Polygon(new double[] { -5, -5, 5, 5, -5 }, new double[] { -5, 5, 5, -5, -5 } );
-    // Polygon polygons[] = new Polygon[] { outer, island };
-    
-    Polygon polygon = new Polygon(new double[] { 0, -50, -50, 50, 50, -50, 0, -5, -5, 5, 5, -5, 0, -10, -10, 10, 10, -10, 0 },
-                                  new double[] { 0, -50, 50, 50, -50, -50, 0, -5, 5, 5, -5, -5, 0, -10, 10, 10, -10, -10, 0 });
-    
-    assertTrue(polygon.contains(-2, 2)); // on the island
-    assertFalse(polygon.contains(-6, 6)); // in the hole
-    assertTrue(polygon.contains(-25, 25)); // on the mainland
-    assertFalse(polygon.contains(-51, 51)); // in the ocean
-  }
-  
-  /** Tests current impl against original algorithm */
-  public void testContainsAgainstOriginal() {
-    for (int i = 0; i < 1000; i++) {
-      Polygon polygon = nextPolygon();
-      // currently we don't generate these, but this test does not want holes.
-      while (polygon.getHoles().length > 0) {
-        polygon = nextPolygon();
-      }
-      
-      double polyLats[] = polygon.getPolyLats();
-      double polyLons[] = polygon.getPolyLons();
-      
-      // random lat/lons against polygon
-      for (int j = 0; j < 1000; j++) {
-        double point[] = GeoTestUtil.nextPointNear(polygon);
-        double latitude = point[0];
-        double longitude = point[1];
-        // bounding box check required due to rounding errors (we don't solve that problem)
-        if (latitude >= polygon.minLat && latitude <= polygon.maxLat && longitude >= polygon.minLon && longitude <= polygon.maxLon) {
-          boolean expected = containsOriginal(polyLats, polyLons, latitude, longitude);
-          assertEquals(expected, polygon.contains(latitude, longitude));
-        }
-      }
-    }
-  }
-  
-  // direct port of PNPOLY C code (https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html)
-  // this allows us to improve the code yet still ensure we have its properties
-  // it is under the BSD license (https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html#License%20to%20Use)
-  //
-  // Copyright (c) 1970-2003, Wm. Randolph Franklin
-  //
-  // Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated 
-  // documentation files (the "Software"), to deal in the Software without restriction, including without limitation 
-  // the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and 
-  // to permit persons to whom the Software is furnished to do so, subject to the following conditions:
-  //
-  // 1. Redistributions of source code must retain the above copyright 
-  //    notice, this list of conditions and the following disclaimers.
-  // 2. Redistributions in binary form must reproduce the above copyright 
-  //    notice in the documentation and/or other materials provided with 
-  //    the distribution.
-  // 3. The name of W. Randolph Franklin may not be used to endorse or 
-  //    promote products derived from this Software without specific 
-  //    prior written permission. 
-  //
-  // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED 
-  // TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL 
-  // THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF 
-  // CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS 
-  // IN THE SOFTWARE. 
-  private static boolean containsOriginal(double polyLats[], double polyLons[], double latitude, double longitude) {
-    boolean c = false;
-    int i, j;
-    int nvert = polyLats.length;
-    double verty[] = polyLats;
-    double vertx[] = polyLons;
-    double testy = latitude;
-    double testx = longitude;
-    for (i = 0, j = nvert-1; i < nvert; j = i++) {
-      if ( ((verty[i]>testy) != (verty[j]>testy)) &&
-     (testx < (vertx[j]-vertx[i]) * (testy-verty[i]) / (verty[j]-verty[i]) + vertx[i]) )
-         c = !c;
-    }
-    return c;
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/837264a4/lucene/core/src/test/org/apache/lucene/geo/TestPolygon2D.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/geo/TestPolygon2D.java b/lucene/core/src/test/org/apache/lucene/geo/TestPolygon2D.java
new file mode 100644
index 0000000..70281ca
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/geo/TestPolygon2D.java
@@ -0,0 +1,289 @@
+/*
+ * 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.geo;
+
+import static org.apache.lucene.geo.GeoTestUtil.nextLatitude;
+import static org.apache.lucene.geo.GeoTestUtil.nextLongitude;
+import static org.apache.lucene.geo.GeoTestUtil.nextPolygon;
+
+import org.apache.lucene.index.PointValues.Relation;
+import org.apache.lucene.util.LuceneTestCase;
+
+/** Test Polygon2D impl */
+public class TestPolygon2D extends LuceneTestCase {
+  
+  /** Three boxes, an island inside a hole inside a shape */
+  public void testMultiPolygon() {
+    Polygon hole = new Polygon(new double[] { -10, -10, 10, 10, -10 }, new double[] { -10, 10, 10, -10, -10 });
+    Polygon outer = new Polygon(new double[] { -50, -50, 50, 50, -50 }, new double[] { -50, 50, 50, -50, -50 }, hole);
+    Polygon island = new Polygon(new double[] { -5, -5, 5, 5, -5 }, new double[] { -5, 5, 5, -5, -5 } );
+    Polygon2D polygon = Polygon2D.create(outer, island);
+    
+    // contains(point)
+    assertTrue(polygon.contains(-2, 2)); // on the island
+    assertFalse(polygon.contains(-6, 6)); // in the hole
+    assertTrue(polygon.contains(-25, 25)); // on the mainland
+    assertFalse(polygon.contains(-51, 51)); // in the ocean
+    
+    // relate(box): this can conservatively return CELL_CROSSES_QUERY
+    assertEquals(Relation.CELL_INSIDE_QUERY, polygon.relate(-2, 2, -2, 2)); // on the island
+    assertEquals(Relation.CELL_OUTSIDE_QUERY, polygon.relate(6, 7, 6, 7)); // in the hole
+    assertEquals(Relation.CELL_INSIDE_QUERY, polygon.relate(24, 25, 24, 25)); // on the mainland
+    assertEquals(Relation.CELL_OUTSIDE_QUERY, polygon.relate(51, 52, 51, 52)); // in the ocean
+    assertEquals(Relation.CELL_CROSSES_QUERY, polygon.relate(-60, 60, -60, 60)); // enclosing us completely
+    assertEquals(Relation.CELL_CROSSES_QUERY, polygon.relate(49, 51, 49, 51)); // overlapping the mainland
+    assertEquals(Relation.CELL_CROSSES_QUERY, polygon.relate(9, 11, 9, 11)); // overlapping the hole
+    assertEquals(Relation.CELL_CROSSES_QUERY, polygon.relate(5, 6, 5, 6)); // overlapping the island
+  }
+  
+  public void testPacMan() throws Exception {
+    // pacman
+    double[] px = {0, 10, 10, 0, -8, -10, -8, 0, 10, 10, 0};
+    double[] py = {0, 5, 9, 10, 9, 0, -9, -10, -9, -5, 0};
+
+    // candidate crosses cell
+    double xMin = 2;//-5;
+    double xMax = 11;//0.000001;
+    double yMin = -1;//0;
+    double yMax = 1;//5;
+
+    // test cell crossing poly
+    Polygon2D polygon = Polygon2D.create(new Polygon(py, px));
+    assertEquals(Relation.CELL_CROSSES_QUERY, polygon.relate(yMin, yMax, xMin, xMax));
+  }
+  
+  public void testBoundingBox() throws Exception {
+    for (int i = 0; i < 100; i++) {
+      Polygon2D polygon = Polygon2D.create(nextPolygon());
+      
+      for (int j = 0; j < 100; j++) {
+        double latitude = nextLatitude();
+        double longitude = nextLongitude();
+        // if the point is within poly, then it should be in our bounding box
+        if (polygon.contains(latitude, longitude)) {
+          assertTrue(latitude >= polygon.minLat && latitude <= polygon.maxLat);
+          assertTrue(longitude >= polygon.minLon && longitude <= polygon.maxLon);
+        }
+      }
+    }
+  }
+  
+  // targets the bounding box directly
+  public void testBoundingBoxEdgeCases() throws Exception {
+    for (int i = 0; i < 100; i++) {
+      Polygon polygon = nextPolygon();
+      Polygon2D impl = Polygon2D.create(polygon);
+      
+      for (int j = 0; j < 100; j++) {
+        double point[] = GeoTestUtil.nextPointNear(polygon);
+        double latitude = point[0];
+        double longitude = point[1];
+        // if the point is within poly, then it should be in our bounding box
+        if (impl.contains(latitude, longitude)) {
+          assertTrue(latitude >= polygon.minLat && latitude <= polygon.maxLat);
+          assertTrue(longitude >= polygon.minLon && longitude <= polygon.maxLon);
+        }
+      }
+    }
+  }
+  
+  /** If polygon.contains(box) returns true, then any point in that box should return true as well */
+  public void testContainsRandom() throws Exception {
+    for (int i = 0; i < 1000; i++) {
+      Polygon polygon = nextPolygon();
+      Polygon2D impl = Polygon2D.create(polygon);
+      
+      for (int j = 0; j < 100; j++) {
+        Rectangle rectangle = GeoTestUtil.nextBoxNear(polygon);
+        // allowed to conservatively return false
+        if (impl.relate(rectangle.minLat, rectangle.maxLat, rectangle.minLon, rectangle.maxLon) == Relation.CELL_INSIDE_QUERY) {
+          for (int k = 0; k < 500; k++) {
+            // this tests in our range but sometimes outside! so we have to double-check its really in other box
+            double point[] = GeoTestUtil.nextPointNear(rectangle);
+            double latitude = point[0];
+            double longitude = point[1];
+            // check for sure its in our box
+            if (latitude >= rectangle.minLat && latitude <= rectangle.maxLat && longitude >= rectangle.minLon && longitude <= rectangle.maxLon) {
+              assertTrue(impl.contains(latitude, longitude));
+            }
+          }
+          for (int k = 0; k < 100; k++) {
+            // this tests in our range but sometimes outside! so we have to double-check its really in other box
+            double point[] = GeoTestUtil.nextPointNear(polygon);
+            double latitude = point[0];
+            double longitude = point[1];
+            // check for sure its in our box
+            if (latitude >= rectangle.minLat && latitude <= rectangle.maxLat && longitude >= rectangle.minLon && longitude <= rectangle.maxLon) {
+              assertTrue(impl.contains(latitude, longitude));
+            }
+          }
+        }
+      }
+    }
+  }
+  
+  /** If polygon.contains(box) returns true, then any point in that box should return true as well */
+  // different from testContainsRandom in that its not a purely random test. we iterate the vertices of the polygon
+  // and generate boxes near each one of those to try to be more efficient.
+  public void testContainsEdgeCases() throws Exception {
+    for (int i = 0; i < 1000; i++) {
+      Polygon polygon = nextPolygon();
+      Polygon2D impl = Polygon2D.create(polygon);
+
+      for (int j = 0; j < 10; j++) {
+        Rectangle rectangle = GeoTestUtil.nextBoxNear(polygon);
+        // allowed to conservatively return false
+        if (impl.relate(rectangle.minLat, rectangle.maxLat, rectangle.minLon, rectangle.maxLon) == Relation.CELL_INSIDE_QUERY) {
+          for (int k = 0; k < 100; k++) {
+            // this tests in our range but sometimes outside! so we have to double-check its really in other box
+            double point[] = GeoTestUtil.nextPointNear(rectangle);
+            double latitude = point[0];
+            double longitude = point[1];
+            // check for sure its in our box
+            if (latitude >= rectangle.minLat && latitude <= rectangle.maxLat && longitude >= rectangle.minLon && longitude <= rectangle.maxLon) {
+              assertTrue(impl.contains(latitude, longitude));
+            }
+          }
+          for (int k = 0; k < 20; k++) {
+            // this tests in our range but sometimes outside! so we have to double-check its really in other box
+            double point[] = GeoTestUtil.nextPointNear(polygon);
+            double latitude = point[0];
+            double longitude = point[1];
+            // check for sure its in our box
+            if (latitude >= rectangle.minLat && latitude <= rectangle.maxLat && longitude >= rectangle.minLon && longitude <= rectangle.maxLon) {
+              assertTrue(impl.contains(latitude, longitude));
+            }
+          }
+        }
+      }
+    }
+  }
+  
+  /** If polygon.intersects(box) returns false, then any point in that box should return false as well */
+  public void testIntersectRandom() {
+    for (int i = 0; i < 100; i++) {
+      Polygon polygon = nextPolygon();
+      Polygon2D impl = Polygon2D.create(polygon);
+      
+      for (int j = 0; j < 100; j++) {
+        Rectangle rectangle = GeoTestUtil.nextBoxNear(polygon);
+        // allowed to conservatively return true.
+        if (impl.relate(rectangle.minLat, rectangle.maxLat, rectangle.minLon, rectangle.maxLon) == Relation.CELL_OUTSIDE_QUERY) {
+          for (int k = 0; k < 1000; k++) {
+            double point[] = GeoTestUtil.nextPointNear(rectangle);
+            // this tests in our range but sometimes outside! so we have to double-check its really in other box
+            double latitude = point[0];
+            double longitude = point[1];
+            // check for sure its in our box
+            if (latitude >= rectangle.minLat && latitude <= rectangle.maxLat && longitude >= rectangle.minLon && longitude <= rectangle.maxLon) {
+              assertFalse(impl.contains(latitude, longitude));
+            }
+          }
+          for (int k = 0; k < 100; k++) {
+            double point[] = GeoTestUtil.nextPointNear(polygon);
+            // this tests in our range but sometimes outside! so we have to double-check its really in other box
+            double latitude = point[0];
+            double longitude = point[1];
+            // check for sure its in our box
+            if (latitude >= rectangle.minLat && latitude <= rectangle.maxLat && longitude >= rectangle.minLon && longitude <= rectangle.maxLon) {
+              assertFalse(impl.contains(latitude, longitude));
+            }
+          }
+        }
+      }
+    }
+  }
+  
+  /** If polygon.intersects(box) returns false, then any point in that box should return false as well */
+  // different from testIntersectsRandom in that its not a purely random test. we iterate the vertices of the polygon
+  // and generate boxes near each one of those to try to be more efficient.
+  public void testIntersectEdgeCases() {
+    for (int i = 0; i < 100; i++) {
+      Polygon polygon = nextPolygon();
+      Polygon2D impl = Polygon2D.create(polygon);
+
+      for (int j = 0; j < 10; j++) {
+        Rectangle rectangle = GeoTestUtil.nextBoxNear(polygon);
+        // allowed to conservatively return false.
+        if (impl.relate(rectangle.minLat, rectangle.maxLat, rectangle.minLon, rectangle.maxLon) == Relation.CELL_OUTSIDE_QUERY) {
+          for (int k = 0; k < 100; k++) {
+            // this tests in our range but sometimes outside! so we have to double-check its really in other box
+            double point[] = GeoTestUtil.nextPointNear(rectangle);
+            double latitude = point[0];
+            double longitude = point[1];
+            // check for sure its in our box
+            if (latitude >= rectangle.minLat && latitude <= rectangle.maxLat && longitude >= rectangle.minLon && longitude <= rectangle.maxLon) {
+              assertFalse(impl.contains(latitude, longitude));
+            }
+          }
+          for (int k = 0; k < 50; k++) {
+            // this tests in our range but sometimes outside! so we have to double-check its really in other box
+            double point[] = GeoTestUtil.nextPointNear(polygon);
+            double latitude = point[0];
+            double longitude = point[1];
+            // check for sure its in our box
+            if (latitude >= rectangle.minLat && latitude <= rectangle.maxLat && longitude >= rectangle.minLon && longitude <= rectangle.maxLon) {
+              assertFalse(impl.contains(latitude, longitude));
+            }
+          }
+        }
+      }
+    }
+  }
+  
+  /** Tests edge case behavior with respect to insideness */
+  public void testEdgeInsideness() {
+    Polygon2D poly = Polygon2D.create(new Polygon(new double[] { -2, -2, 2, 2, -2 }, new double[] { -2, 2, 2, -2, -2 }));
+    assertTrue(poly.contains(-2, -2)); // bottom left corner: true
+    assertFalse(poly.contains(-2, 2));  // bottom right corner: false
+    assertFalse(poly.contains(2, -2));  // top left corner: false
+    assertFalse(poly.contains(2,  2));  // top right corner: false
+    assertTrue(poly.contains(-2, -1)); // bottom side: true
+    assertTrue(poly.contains(-2, 0));  // bottom side: true
+    assertTrue(poly.contains(-2, 1));  // bottom side: true
+    assertFalse(poly.contains(2, -1));  // top side: false
+    assertFalse(poly.contains(2, 0));   // top side: false
+    assertFalse(poly.contains(2, 1));   // top side: false
+    assertFalse(poly.contains(-1, 2));  // right side: false
+    assertFalse(poly.contains(0, 2));   // right side: false
+    assertFalse(poly.contains(1, 2));   // right side: false
+    assertTrue(poly.contains(-1, -2)); // left side: true
+    assertTrue(poly.contains(0, -2));  // left side: true
+    assertTrue(poly.contains(1, -2));  // left side: true
+  }
+  
+  /** Tests current impl against original algorithm */
+  public void testContainsAgainstOriginal() {
+    for (int i = 0; i < 1000; i++) {
+      Polygon polygon = nextPolygon();
+      // currently we don't generate these, but this test does not want holes.
+      while (polygon.getHoles().length > 0) {
+        polygon = nextPolygon();
+      }
+      Polygon2D impl = Polygon2D.create(polygon);
+      
+      // random lat/lons against polygon
+      for (int j = 0; j < 1000; j++) {
+        double point[] = GeoTestUtil.nextPointNear(polygon);
+        double latitude = point[0];
+        double longitude = point[1];
+        boolean expected = GeoTestUtil.containsSlowly(polygon, latitude, longitude);
+        assertEquals(expected, impl.contains(latitude, longitude));
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/837264a4/lucene/sandbox/src/java/org/apache/lucene/document/LatLonGrid.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonGrid.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonGrid.java
deleted file mode 100644
index 4b3b2b2..0000000
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonGrid.java
+++ /dev/null
@@ -1,168 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.document;
-
-import org.apache.lucene.geo.Polygon;
-import org.apache.lucene.index.PointValues.Relation;
-import org.apache.lucene.util.FixedBitSet;
-
-import static org.apache.lucene.geo.GeoEncodingUtils.decodeLatitude;
-import static org.apache.lucene.geo.GeoEncodingUtils.decodeLongitude;
-
-/**
- * This is a temporary hack, until some polygon methods have better performance!
- * <p>
- * When this file is removed then we have made good progress! In general we don't call
- * the point-in-polygon algorithm that much, because of how BKD divides up the data. But
- * today the method is very slow (general to all polygons, linear with the number of vertices).
- * At the same time polygon-rectangle relation operations are also slow in the same way, this
- * just really ensures they are the bottleneck by removing most of the point-in-polygon calls.
- * <p>
- * See the "grid" algorithm description here: http://erich.realtimerendering.com/ptinpoly/
- * A few differences:
- * <ul>
- *   <li> We work in an integer encoding, so edge cases are simpler.
- *   <li> We classify each grid cell as "contained", "not contained", or "don't know".
- *   <li> We form a grid over a potentially complex multipolygon with holes.
- *   <li> Construction is less efficient because we do not do anything "smart" such
- *        as following polygon edges. 
- *   <li> Instead we construct a baby tree to reduce the number of relation operations,
- *        which are currently expensive.
- * </ul>
- */
-// TODO: just make a more proper tree (maybe in-ram BKD)? then we can answer most 
-// relational operations as rectangle <-> rectangle relations in integer space in log(n) time..
-final class LatLonGrid {
-  // must be a power of two!
-  static final int GRID_SIZE = 1<<7;
-  final int minLat;
-  final int maxLat;
-  final int minLon;
-  final int maxLon;
-  // TODO: something more efficient than parallel bitsets? maybe one bitset?
-  final FixedBitSet haveAnswer = new FixedBitSet(GRID_SIZE * GRID_SIZE);
-  final FixedBitSet answer = new FixedBitSet(GRID_SIZE * GRID_SIZE);
-  
-  final long latPerCell;
-  final long lonPerCell;
-  
-  final LatLonTree[] tree;
-  
-  LatLonGrid(int minLat, int maxLat, int minLon, int maxLon, LatLonTree[] tree) {
-    this.minLat = minLat;
-    this.maxLat = maxLat;
-    this.minLon = minLon;
-    this.maxLon = maxLon;
-    this.tree = tree;
-    if (minLon > maxLon) {
-      // maybe make 2 grids if you want this? 
-      throw new IllegalArgumentException("Grid cannot cross the dateline");
-    }
-    if (minLat > maxLat) {
-      throw new IllegalArgumentException("bogus grid");
-    }
-    long latitudeRange = maxLat - (long) minLat;
-    long longitudeRange = maxLon - (long) minLon;
-
-    // if the range is too small, we can't divide it up in our grid nicely.
-    // in this case of a tiny polygon, we just make an empty grid instead of complicating/slowing down code.
-    final long minRange = (GRID_SIZE - 1) * (GRID_SIZE - 1);
-    if (latitudeRange < minRange || longitudeRange < minRange) {
-      latPerCell = lonPerCell = Long.MAX_VALUE;
-    } else {
-      // we spill over the edge of the bounding box in each direction a bit,
-      // but it prevents edge case bugs.
-      latPerCell = latitudeRange / (GRID_SIZE - 1);
-      lonPerCell = longitudeRange / (GRID_SIZE - 1);
-      fill(0, GRID_SIZE, 0, GRID_SIZE);
-    }
-  }
-  
-  /** fills a 2D range of grid cells [minLatIndex .. maxLatIndex) X [minLonIndex .. maxLonIndex) */
-  void fill(int minLatIndex, int maxLatIndex, int minLonIndex, int maxLonIndex) {
-    // grid cells at the edge of the bounding box are typically smaller than normal, because we spill over.
-    long cellMinLat = minLat + (minLatIndex * latPerCell);
-    long cellMaxLat = Math.min(maxLat, minLat + (maxLatIndex * latPerCell) - 1);
-    long cellMinLon = minLon + (minLonIndex * lonPerCell);
-    long cellMaxLon = Math.min(maxLon, minLon + (maxLonIndex * lonPerCell) - 1);
-
-    assert cellMinLat <= maxLat && cellMinLon <= maxLon;
-    assert cellMaxLat >= cellMinLat;
-    assert cellMaxLon >= cellMinLon;
-
-    Relation relation = LatLonTree.relate(tree, decodeLatitude((int) cellMinLat),
-                                                decodeLatitude((int) cellMaxLat),
-                                                decodeLongitude((int) cellMinLon),
-                                                decodeLongitude((int) cellMaxLon));
-    if (relation != Relation.CELL_CROSSES_QUERY) {
-      // we know the answer for this region, fill the cell range
-      for (int i = minLatIndex; i < maxLatIndex; i++) {
-        for (int j = minLonIndex; j < maxLonIndex; j++) {
-          int index = i * GRID_SIZE + j;
-          assert haveAnswer.get(index) == false;
-          haveAnswer.set(index);
-          if (relation == Relation.CELL_INSIDE_QUERY) {
-            answer.set(index);
-          }
-        }
-      }
-    } else if (minLatIndex == maxLatIndex - 1) {
-      // nothing more to do: this is a single grid cell (leaf node) and
-      // is an edge case for the polygon.
-    } else {
-      // grid range crosses our polygon, keep recursing.
-      int midLatIndex = (minLatIndex + maxLatIndex) >>> 1;
-      int midLonIndex = (minLonIndex + maxLonIndex) >>> 1;
-      fill(minLatIndex, midLatIndex, minLonIndex, midLonIndex);
-      fill(minLatIndex, midLatIndex, midLonIndex, maxLonIndex);
-      fill(midLatIndex, maxLatIndex, minLonIndex, midLonIndex);
-      fill(midLatIndex, maxLatIndex, midLonIndex, maxLonIndex);
-    }
-  }
-  
-  /** Returns true if inside one of our polygons, false otherwise */
-  boolean contains(int latitude, int longitude) {
-    // first see if the grid knows the answer
-    int index = index(latitude, longitude);
-    if (index == -1) {
-      return false; // outside of bounding box range
-    } else if (haveAnswer.get(index)) {
-      return answer.get(index);
-    }
-
-    // the grid is unsure (boundary): do a real test.
-    double docLatitude = decodeLatitude(latitude);
-    double docLongitude = decodeLongitude(longitude);
-    return LatLonTree.contains(tree, docLatitude, docLongitude);
-  }
-  
-  /** Returns grid index of lat/lon, or -1 if the value is outside of the bounding box. */
-  private int index(int latitude, int longitude) {
-    if (latitude < minLat || latitude > maxLat || longitude < minLon || longitude > maxLon) {
-      return -1; // outside of bounding box range
-    }
-    
-    long latRel = latitude - (long) minLat;
-    long lonRel = longitude - (long) minLon;
-    
-    int latIndex = (int) (latRel / latPerCell);
-    assert latIndex < GRID_SIZE;
-    int lonIndex = (int) (lonRel / lonPerCell);
-    assert lonIndex < GRID_SIZE;
-    return latIndex * GRID_SIZE + lonIndex;
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/837264a4/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java
index e68cb45..ee7c1e8 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java
@@ -38,6 +38,7 @@ import org.apache.lucene.util.DocIdSetBuilder;
 import org.apache.lucene.util.NumericUtils;
 import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.geo.Polygon;
+import org.apache.lucene.geo.Polygon2D;
 
 import static org.apache.lucene.geo.GeoEncodingUtils.decodeLatitude;
 import static org.apache.lucene.geo.GeoEncodingUtils.decodeLongitude;
@@ -92,11 +93,7 @@ final class LatLonPointInPolygonQuery extends Query {
     NumericUtils.intToSortableBytes(encodeLongitude(box.minLon), minLon, 0);
     NumericUtils.intToSortableBytes(encodeLongitude(box.maxLon), maxLon, 0);
 
-    final LatLonTree[] tree = LatLonTree.build(polygons);
-    final LatLonGrid grid = new LatLonGrid(encodeLatitude(box.minLat),
-                                           encodeLatitude(box.maxLat),
-                                           encodeLongitude(box.minLon),
-                                           encodeLongitude(box.maxLon), tree);
+    final Polygon2D tree = Polygon2D.create(polygons);
 
     return new ConstantScoreWeight(this) {
 
@@ -132,17 +129,8 @@ final class LatLonPointInPolygonQuery extends Query {
 
                            @Override
                            public void visit(int docID, byte[] packedValue) {
-                             // we bounds check individual values, as subtrees may cross, but we are being sent the values anyway:
-                             // this reduces the amount of docvalues fetches (improves approximation)
-                             if (StringHelper.compare(Integer.BYTES, packedValue, 0, maxLat, 0) > 0 ||
-                                 StringHelper.compare(Integer.BYTES, packedValue, 0, minLat, 0) < 0 ||
-                                 StringHelper.compare(Integer.BYTES, packedValue, Integer.BYTES, maxLon, 0) > 0 ||
-                                 StringHelper.compare(Integer.BYTES, packedValue, Integer.BYTES, minLon, 0) < 0) {
-                               // outside of global bounding box range
-                               return;
-                             }
-                             if (grid.contains(NumericUtils.sortableBytesToInt(packedValue, 0), 
-                                               NumericUtils.sortableBytesToInt(packedValue, Integer.BYTES))) {
+                             if (tree.contains(decodeLatitude(packedValue, 0), 
+                                               decodeLongitude(packedValue, Integer.BYTES))) {
                                result.add(docID);
                              }
                            }
@@ -162,7 +150,7 @@ final class LatLonPointInPolygonQuery extends Query {
                              double cellMaxLat = decodeLatitude(maxPackedValue, 0);
                              double cellMaxLon = decodeLongitude(maxPackedValue, Integer.BYTES);
 
-                             return LatLonTree.relate(tree, cellMinLat, cellMaxLat, cellMinLon, cellMaxLon);
+                             return tree.relate(cellMinLat, cellMaxLat, cellMinLon, cellMaxLon);
                            }
                          });
 


[08/50] [abbrv] lucene-solr:solr-5750: Merge remote-tracking branch 'origin/master'

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

Branch: refs/heads/solr-5750
Commit: dac6fe09bb532331316686eed417031ebc96c1f4
Parents: e8cc19e 2ee8426
Author: Noble Paul <no...@apache.org>
Authored: Thu Apr 21 20:38:43 2016 +0530
Committer: Noble Paul <no...@apache.org>
Committed: Thu Apr 21 20:38:43 2016 +0530

----------------------------------------------------------------------
 lucene/CHANGES.txt                                                | 3 +++
 .../src/java/org/apache/lucene/index/memory/MemoryIndex.java      | 1 +
 2 files changed, 4 insertions(+)
----------------------------------------------------------------------



[14/50] [abbrv] lucene-solr:solr-5750: LUCENE-7243: Removed the LeafReaderContext parameter from QueryCachingPolicy#shouldCache.

Posted by ds...@apache.org.
LUCENE-7243: Removed the LeafReaderContext parameter from QueryCachingPolicy#shouldCache.


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

Branch: refs/heads/solr-5750
Commit: 927a44881c020efd6fa79ad5633c8e96bfa716df
Parents: 7b89d0f
Author: Adrien Grand <jp...@gmail.com>
Authored: Fri Apr 22 13:08:48 2016 +0200
Committer: Adrien Grand <jp...@gmail.com>
Committed: Fri Apr 22 13:09:26 2016 +0200

----------------------------------------------------------------------
 lucene/CHANGES.txt                                |  3 +++
 .../org/apache/lucene/search/LRUQueryCache.java   |  4 ++--
 .../apache/lucene/search/QueryCachingPolicy.java  | 18 ++++++++----------
 .../search/UsageTrackingQueryCachingPolicy.java   |  7 +------
 .../apache/lucene/search/TestIndexSearcher.java   |  2 +-
 .../apache/lucene/search/TestLRUQueryCache.java   |  8 ++++----
 .../TestUsageTrackingFilterCachingPolicy.java     | 17 +----------------
 .../org/apache/lucene/queries/TermsQueryTest.java | 16 ++--------------
 .../org/apache/lucene/util/LuceneTestCase.java    |  2 +-
 9 files changed, 23 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/927a4488/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index c2561b3..18409c9 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -40,6 +40,9 @@ API Changes
 * LUCENE-7150: Spatial3d gets useful APIs to create common shape
   queries, matching LatLonPoint.  (Karl Wright via Mike McCandless)
 
+* LUCENE-7243: Removed the LeafReaderContext parameter from
+  QueryCachingPolicy#shouldCache. (Adrien Grand)
+
 Optimizations
 
 * LUCENE-7071: Reduce bytes copying in OfflineSorter, giving ~10%

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/927a4488/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java b/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
index f4cf8dc..15c0f2b 100644
--- a/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
+++ b/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
@@ -661,7 +661,7 @@ public class LRUQueryCache implements QueryCache, Accountable {
 
       DocIdSet docIdSet = get(in.getQuery(), context);
       if (docIdSet == null) {
-        if (policy.shouldCache(in.getQuery(), context)) {
+        if (policy.shouldCache(in.getQuery())) {
           docIdSet = cache(context);
           putIfAbsent(in.getQuery(), context, docIdSet);
         } else {
@@ -694,7 +694,7 @@ public class LRUQueryCache implements QueryCache, Accountable {
 
       DocIdSet docIdSet = get(in.getQuery(), context);
       if (docIdSet == null) {
-        if (policy.shouldCache(in.getQuery(), context)) {
+        if (policy.shouldCache(in.getQuery())) {
           docIdSet = cache(context);
           putIfAbsent(in.getQuery(), context, docIdSet);
         } else {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/927a4488/lucene/core/src/java/org/apache/lucene/search/QueryCachingPolicy.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/QueryCachingPolicy.java b/lucene/core/src/java/org/apache/lucene/search/QueryCachingPolicy.java
index c0f6aa8..fabd971 100644
--- a/lucene/core/src/java/org/apache/lucene/search/QueryCachingPolicy.java
+++ b/lucene/core/src/java/org/apache/lucene/search/QueryCachingPolicy.java
@@ -19,8 +19,6 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
-import org.apache.lucene.index.LeafReaderContext;
-
 /**
  * A policy defining which filters should be cached.
  *
@@ -40,7 +38,7 @@ public interface QueryCachingPolicy {
     public void onUse(Query query) {}
 
     @Override
-    public boolean shouldCache(Query query, LeafReaderContext context) throws IOException {
+    public boolean shouldCache(Query query) throws IOException {
       return true;
     }
 
@@ -51,12 +49,12 @@ public interface QueryCachingPolicy {
    *  in order to make decisions. */
   void onUse(Query query);
 
-  /** Whether the given {@link DocIdSet} should be cached on a given segment.
-   *  This method will be called on each leaf context to know if the filter
-   *  should be cached on this particular leaf. The filter cache will first
-   *  attempt to load a {@link DocIdSet} from the cache. If it is not cached
-   *  yet and this method returns <tt>true</tt> then a cache entry will be
-   *  generated. Otherwise an uncached set will be returned. */
-  boolean shouldCache(Query query, LeafReaderContext context) throws IOException;
+  /** Whether the given {@link Query} is worth caching.
+   *  This method will be called by the {@link QueryCache} to know whether to
+   *  cache. It will first attempt to load a {@link DocIdSet} from the cache.
+   *  If it is not cached yet and this method returns <tt>true</tt> then a
+   *  cache entry will be generated. Otherwise an uncached scorer will be
+   *  returned. */
+  boolean shouldCache(Query query) throws IOException;
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/927a4488/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java b/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java
index 4eb9289..ab68eeb 100644
--- a/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java
+++ b/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java
@@ -19,17 +19,12 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.util.FrequencyTrackingRingBuffer;
 
 /**
  * A {@link QueryCachingPolicy} that tracks usage statistics of recently-used
  * filters in order to decide on which filters are worth caching.
  *
- * It also uses some heuristics on segments, filters and the doc id sets that
- * they produce in order to cache more aggressively when the execution cost
- * significantly outweighs the caching overhead.
- *
  * @lucene.experimental
  */
 public final class UsageTrackingQueryCachingPolicy implements QueryCachingPolicy {
@@ -128,7 +123,7 @@ public final class UsageTrackingQueryCachingPolicy implements QueryCachingPolicy
   }
 
   @Override
-  public boolean shouldCache(Query query, LeafReaderContext context) throws IOException {
+  public boolean shouldCache(Query query) throws IOException {
     if (query instanceof MatchAllDocsQuery
         // MatchNoDocsQuery currently rewrites to a BooleanQuery,
         // but who knows, it might get its own Weight one day

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/927a4488/lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java b/lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java
index e7cdfcd..8201bf8 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java
@@ -207,7 +207,7 @@ public class TestIndexSearcher extends LuceneTestCase {
     assertEquals(IndexSearcher.getDefaultQueryCachingPolicy(), searcher.getQueryCachingPolicy());
     QueryCachingPolicy dummyPolicy = new QueryCachingPolicy() {
       @Override
-      public boolean shouldCache(Query query, LeafReaderContext context) throws IOException {
+      public boolean shouldCache(Query query) throws IOException {
         return false;
       }
       @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/927a4488/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
index 2552b04..63ccdd8 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
@@ -65,7 +65,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
     public void onUse(Query query) {}
 
     @Override
-    public boolean shouldCache(Query query, LeafReaderContext context) throws IOException {
+    public boolean shouldCache(Query query) throws IOException {
       return false;
     }
 
@@ -455,7 +455,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
     final QueryCachingPolicy countingPolicy = new QueryCachingPolicy() {
 
       @Override
-      public boolean shouldCache(Query query, LeafReaderContext context) throws IOException {
+      public boolean shouldCache(Query query) throws IOException {
         return random().nextBoolean();
       }
 
@@ -762,7 +762,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
     final QueryCachingPolicy policy = new QueryCachingPolicy() {
 
       @Override
-      public boolean shouldCache(Query query, LeafReaderContext context) throws IOException {
+      public boolean shouldCache(Query query) throws IOException {
         assertEquals(expectedCacheKey, query);
         return true;
       }
@@ -1080,7 +1080,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
     }
 
     @Override
-    public boolean shouldCache(Query query, LeafReaderContext context) throws IOException {
+    public boolean shouldCache(Query query) throws IOException {
       return true;
     }
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/927a4488/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java b/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java
index c656b85..29ed22f 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java
@@ -16,15 +16,8 @@
  */
 package org.apache.lucene.search;
 
-import org.apache.lucene.document.Document;
 import org.apache.lucene.document.IntPoint;
-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.MultiReader;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
 
 public class TestUsageTrackingFilterCachingPolicy extends LuceneTestCase {
@@ -41,15 +34,7 @@ public class TestUsageTrackingFilterCachingPolicy extends LuceneTestCase {
     for (int i = 0; i < 1000; ++i) {
       policy.onUse(q);
     }
-    Directory dir = newDirectory();
-    IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null));
-    w.addDocument(new Document());
-    IndexReader r = DirectoryReader.open(w);
-    assertFalse(policy.shouldCache(q, getOnlyLeafReader(r).getContext()));
-    
-    r.close();
-    w.close();
-    dir.close();
+    assertFalse(policy.shouldCache(q));
   }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/927a4488/lucene/queries/src/test/org/apache/lucene/queries/TermsQueryTest.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/test/org/apache/lucene/queries/TermsQueryTest.java b/lucene/queries/src/test/org/apache/lucene/queries/TermsQueryTest.java
index a87e45d..71eeef4 100644
--- a/lucene/queries/src/test/org/apache/lucene/queries/TermsQueryTest.java
+++ b/lucene/queries/src/test/org/apache/lucene/queries/TermsQueryTest.java
@@ -330,24 +330,12 @@ public class TermsQueryTest extends LuceneTestCase {
   }
 
   public void testIsConsideredCostlyByQueryCache() throws IOException {
-    Directory dir = newDirectory();
-    IndexWriterConfig iwc = newIndexWriterConfig();
-    IndexWriter w = new IndexWriter(dir, iwc);
-    Document doc = new Document();
-    for (int i = 0; i < 10000; ++i) {
-      w.addDocument(doc);
-    }
-    w.forceMerge(1);
-    DirectoryReader reader = DirectoryReader.open(w);
-    w.close();
     TermsQuery query = new TermsQuery(new Term("foo", "bar"), new Term("foo", "baz"));
     UsageTrackingQueryCachingPolicy policy = new UsageTrackingQueryCachingPolicy();
-    assertFalse(policy.shouldCache(query, getOnlyLeafReader(reader).getContext()));
+    assertFalse(policy.shouldCache(query));
     policy.onUse(query);
     policy.onUse(query);
     // cached after two uses
-    assertTrue(policy.shouldCache(query, getOnlyLeafReader(reader).getContext()));
-    reader.close();
-    dir.close();
+    assertTrue(policy.shouldCache(query));
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/927a4488/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
index 70ed86d..52aca7e 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
@@ -476,7 +476,7 @@ public abstract class LuceneTestCase extends Assert {
     public void onUse(Query query) {}
 
     @Override
-    public boolean shouldCache(Query query, LeafReaderContext context) throws IOException {
+    public boolean shouldCache(Query query) throws IOException {
       return random().nextBoolean();
     }
 


[02/50] [abbrv] lucene-solr:solr-5750: SOLR-9020: Implement StatementImpl/ResultSetImpl get/set fetch* methods and proper errors for traversal methods

Posted by ds...@apache.org.
SOLR-9020: Implement StatementImpl/ResultSetImpl get/set fetch* methods and proper errors for traversal 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/a9a842f0
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/a9a842f0
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/a9a842f0

Branch: refs/heads/solr-5750
Commit: a9a842f05dcebb6b86a6f47354aa760a83763e6c
Parents: b44ca08
Author: Kevin Risden <kr...@apache.org>
Authored: Wed Apr 20 13:40:59 2016 -0500
Committer: Kevin Risden <kr...@apache.org>
Committed: Wed Apr 20 14:48:23 2016 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  2 +
 .../solr/client/solrj/io/sql/ResultSetImpl.java | 76 +++++++++++++++-----
 .../solr/client/solrj/io/sql/StatementImpl.java | 46 +++++++-----
 .../solr/client/solrj/io/sql/JdbcTest.java      | 53 ++++++++------
 4 files changed, 124 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a9a842f0/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 8630ea4..31ae74c 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -109,6 +109,8 @@ New Features
 
 * SOLR-8809: Implement Connection.prepareStatement (Kevin Risden)
 
+* SOLR-9020: Implement StatementImpl/ResultSetImpl get/set fetch* methods and proper errors for traversal methods (Kevin Risden)
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a9a842f0/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/ResultSetImpl.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/ResultSetImpl.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/ResultSetImpl.java
index e4dcaed..91b99b9 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/ResultSetImpl.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/ResultSetImpl.java
@@ -31,6 +31,7 @@ import java.sql.ResultSet;
 import java.sql.ResultSetMetaData;
 import java.sql.RowId;
 import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
 import java.sql.SQLWarning;
 import java.sql.SQLXML;
 import java.sql.Statement;
@@ -480,92 +481,133 @@ class ResultSetImpl implements ResultSet {
 
   @Override
   public boolean isBeforeFirst() throws SQLException {
-    throw new UnsupportedOperationException();
+    checkClosed();
+
+    throw new SQLFeatureNotSupportedException();
   }
 
   @Override
   public boolean isAfterLast() throws SQLException {
-    throw new UnsupportedOperationException();
+    checkClosed();
+
+    throw new SQLFeatureNotSupportedException();
   }
 
   @Override
   public boolean isFirst() throws SQLException {
-    throw new UnsupportedOperationException();
+    checkClosed();
+
+    throw new SQLFeatureNotSupportedException();
   }
 
   @Override
   public boolean isLast() throws SQLException {
-    throw new UnsupportedOperationException();
+    checkClosed();
+
+    throw new SQLFeatureNotSupportedException();
   }
 
   @Override
   public void beforeFirst() throws SQLException {
-    throw new UnsupportedOperationException();
+    checkClosed();
+
+    throw new SQLException("beforeFirst() not supported on ResultSet with type TYPE_FORWARD_ONLY");
   }
 
   @Override
   public void afterLast() throws SQLException {
-    throw new UnsupportedOperationException();
+    checkClosed();
+
+    throw new SQLException("afterLast() not supported on ResultSet with type TYPE_FORWARD_ONLY");
   }
 
   @Override
   public boolean first() throws SQLException {
-    throw new UnsupportedOperationException();
+    checkClosed();
+
+    throw new SQLException("first() not supported on ResultSet with type TYPE_FORWARD_ONLY");
   }
 
   @Override
   public boolean last() throws SQLException {
-    throw new UnsupportedOperationException();
+    checkClosed();
+
+    throw new SQLException("last() not supported on ResultSet with type TYPE_FORWARD_ONLY");
   }
 
   @Override
   public int getRow() throws SQLException {
-    throw new UnsupportedOperationException();
+    checkClosed();
+
+    throw new SQLFeatureNotSupportedException();
   }
 
   @Override
   public boolean absolute(int row) throws SQLException {
-    throw new UnsupportedOperationException();
+    checkClosed();
+
+    throw new SQLException("absolute() not supported on ResultSet with type TYPE_FORWARD_ONLY");
   }
 
   @Override
   public boolean relative(int rows) throws SQLException {
-    throw new UnsupportedOperationException();
+    checkClosed();
+
+    throw new SQLException("relative() not supported on ResultSet with type TYPE_FORWARD_ONLY");
   }
 
   @Override
   public boolean previous() throws SQLException {
-    throw new UnsupportedOperationException();
+    checkClosed();
+
+    throw new SQLException("previous() not supported on ResultSet with type TYPE_FORWARD_ONLY");
   }
 
   @Override
   public void setFetchDirection(int direction) throws SQLException {
-    throw new UnsupportedOperationException();
+    checkClosed();
+
+    if(direction != ResultSet.FETCH_FORWARD) {
+      throw new SQLException("Direction must be FETCH_FORWARD since ResultSet " +
+          "type is TYPE_FORWARD_ONLY");
+    }
   }
 
   @Override
   public int getFetchDirection() throws SQLException {
-    throw new UnsupportedOperationException();
+    checkClosed();
+
+    return ResultSet.FETCH_FORWARD;
   }
 
   @Override
   public void setFetchSize(int rows) throws SQLException {
-    throw new UnsupportedOperationException();
+    checkClosed();
+
+    if(rows < 0) {
+      throw new SQLException("Rows must be >= 0");
+    }
   }
 
   @Override
   public int getFetchSize() throws SQLException {
-    throw new UnsupportedOperationException();
+    checkClosed();
+
+    return 0;
   }
 
   @Override
   public int getType() throws SQLException {
+    checkClosed();
+
     return ResultSet.TYPE_FORWARD_ONLY;
   }
 
   @Override
   public int getConcurrency() throws SQLException {
-    throw new UnsupportedOperationException();
+    checkClosed();
+
+    return ResultSet.CONCUR_READ_ONLY;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a9a842f0/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/StatementImpl.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/StatementImpl.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/StatementImpl.java
index 24b20a4..1b1200d 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/StatementImpl.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/StatementImpl.java
@@ -51,6 +51,12 @@ class StatementImpl implements Statement {
     this.connection = connection;
   }
 
+  private void checkClosed() throws SQLException {
+    if(isClosed()) {
+      throw new SQLException("Statement is closed.");
+    }
+  }
+
   private ResultSet executeQueryImpl(String sql) throws SQLException {
     try {
       if(this.currentResultSet != null) {
@@ -171,18 +177,14 @@ class StatementImpl implements Statement {
 
   @Override
   public SQLWarning getWarnings() throws SQLException {
-    if(isClosed()) {
-      throw new SQLException("Statement is closed.");
-    }
+    checkClosed();
 
     return this.currentWarning;
   }
 
   @Override
   public void clearWarnings() throws SQLException {
-    if(isClosed()) {
-      throw new SQLException("Statement is closed.");
-    }
+    checkClosed();
 
     this.currentWarning = null;
   }
@@ -212,9 +214,7 @@ class StatementImpl implements Statement {
 
   @Override
   public int getUpdateCount() throws SQLException {
-    if(isClosed()) {
-      throw new SQLException("Statement is closed");
-    }
+    checkClosed();
 
     // TODO Add logic when update statements are added to JDBC.
     return -1;
@@ -222,9 +222,7 @@ class StatementImpl implements Statement {
 
   @Override
   public boolean getMoreResults() throws SQLException {
-    if(isClosed()) {
-      throw new SQLException("Statement is closed");
-    }
+    checkClosed();
 
     // Currently multiple result sets are not possible yet
     this.currentResultSet.close();
@@ -233,32 +231,48 @@ class StatementImpl implements Statement {
 
   @Override
   public void setFetchDirection(int direction) throws SQLException {
-    throw new UnsupportedOperationException();
+    checkClosed();
+
+    if(direction != ResultSet.FETCH_FORWARD) {
+      throw new SQLException("Direction must be ResultSet.FETCH_FORWARD currently");
+    }
   }
 
   @Override
   public int getFetchDirection() throws SQLException {
-    throw new UnsupportedOperationException();
+    checkClosed();
+
+    return ResultSet.FETCH_FORWARD;
   }
 
   @Override
   public void setFetchSize(int rows) throws SQLException {
+    checkClosed();
 
+    if(rows < 0) {
+      throw new SQLException("Rows must be >= 0");
+    }
   }
 
   @Override
   public int getFetchSize() throws SQLException {
+    checkClosed();
+
     return 0;
   }
 
   @Override
   public int getResultSetConcurrency() throws SQLException {
-    throw new UnsupportedOperationException();
+    checkClosed();
+
+    return ResultSet.CONCUR_READ_ONLY;
   }
 
   @Override
   public int getResultSetType() throws SQLException {
-    throw new UnsupportedOperationException();
+    checkClosed();
+
+    return ResultSet.TYPE_FORWARD_ONLY;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a9a842f0/solr/solrj/src/test/org/apache/solr/client/solrj/io/sql/JdbcTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/sql/JdbcTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/sql/JdbcTest.java
index 40d14a1..9050092 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/sql/JdbcTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/sql/JdbcTest.java
@@ -501,16 +501,9 @@ public class JdbcTest extends AbstractFullDistribZkTestBase {
       con.clearWarnings();
       assertNull(con.getWarnings());
 
-      try (Statement statement = con.createStatement()) {
-        assertEquals(con, statement.getConnection());
-
-        assertNull(statement.getWarnings());
-        statement.clearWarnings();
-        assertNull(statement.getWarnings());
 
-        assertEquals(0, statement.getFetchSize());
-        statement.setFetchSize(0);
-        assertEquals(0, statement.getFetchSize());
+      try (Statement statement = con.createStatement()) {
+        checkStatement(con, statement);
 
         try (ResultSet rs = statement.executeQuery(sql)) {
           assertEquals(statement, rs.getStatement());
@@ -533,15 +526,7 @@ public class JdbcTest extends AbstractFullDistribZkTestBase {
       }
 
       try (PreparedStatement statement = con.prepareStatement(sql)) {
-        assertEquals(con, statement.getConnection());
-
-        assertNull(statement.getWarnings());
-        statement.clearWarnings();
-        assertNull(statement.getWarnings());
-
-        assertEquals(0, statement.getFetchSize());
-        statement.setFetchSize(0);
-        assertEquals(0, statement.getFetchSize());
+        checkStatement(con, statement);
 
         try (ResultSet rs = statement.executeQuery()) {
           assertEquals(statement, rs.getStatement());
@@ -565,6 +550,25 @@ public class JdbcTest extends AbstractFullDistribZkTestBase {
     }
   }
 
+  private void checkStatement(Connection con, Statement statement) throws Exception {
+    assertEquals(con, statement.getConnection());
+
+    assertNull(statement.getWarnings());
+    statement.clearWarnings();
+    assertNull(statement.getWarnings());
+
+    assertEquals(ResultSet.TYPE_FORWARD_ONLY, statement.getResultSetType());
+    assertEquals(ResultSet.CONCUR_READ_ONLY, statement.getResultSetConcurrency());
+
+    assertEquals(ResultSet.FETCH_FORWARD, statement.getFetchDirection());
+    statement.setFetchDirection(ResultSet.FETCH_FORWARD);
+    assertEquals(ResultSet.FETCH_FORWARD, statement.getFetchDirection());
+
+    assertEquals(0, statement.getFetchSize());
+    statement.setFetchSize(0);
+    assertEquals(0, statement.getFetchSize());
+  }
+
   private void checkResultSetMetadata(ResultSet rs) throws Exception {
     ResultSetMetaData resultSetMetaData = rs.getMetaData();
 
@@ -604,12 +608,21 @@ public class JdbcTest extends AbstractFullDistribZkTestBase {
   }
 
   private void checkResultSet(ResultSet rs) throws Exception {
-    assertEquals(ResultSet.TYPE_FORWARD_ONLY, rs.getType());
-
     assertNull(rs.getWarnings());
     rs.clearWarnings();
     assertNull(rs.getWarnings());
 
+    assertEquals(ResultSet.TYPE_FORWARD_ONLY, rs.getType());
+    assertEquals(ResultSet.CONCUR_READ_ONLY, rs.getConcurrency());
+
+    assertEquals(ResultSet.FETCH_FORWARD, rs.getFetchDirection());
+    rs.setFetchDirection(ResultSet.FETCH_FORWARD);
+    assertEquals(ResultSet.FETCH_FORWARD, rs.getFetchDirection());
+
+    assertEquals(0, rs.getFetchSize());
+    rs.setFetchSize(10);
+    assertEquals(0, rs.getFetchSize());
+
     assertTrue(rs.next());
 
     assertEquals(14L, rs.getObject("a_i"));


[21/50] [abbrv] lucene-solr:solr-5750: Two nested classes made private that should never have been public.

Posted by ds...@apache.org.
Two nested classes made private that should never have been public.


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

Branch: refs/heads/solr-5750
Commit: 69be7dc2a3ebe9e5170c8d0c5079d863ce73dab6
Parents: 97e8f1a
Author: Karl Wright <Da...@gmail.com>
Authored: Sun Apr 24 02:23:05 2016 -0400
Committer: Karl Wright <Da...@gmail.com>
Committed: Sun Apr 24 02:23:05 2016 -0400

----------------------------------------------------------------------
 .../java/org/apache/lucene/spatial3d/geom/GeoStandardPath.java   | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/69be7dc2/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoStandardPath.java
----------------------------------------------------------------------
diff --git a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoStandardPath.java b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoStandardPath.java
index 312b79e..b24d5af 100755
--- a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoStandardPath.java
+++ b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoStandardPath.java
@@ -336,7 +336,7 @@ class GeoStandardPath extends GeoBasePath {
    *    we generate no circle at all.  If there is one intersection only, then we generate a plane that includes that intersection, as well as the remaining
    *    cutoff plane/edge plane points.
    */
-  public static class SegmentEndpoint {
+  private static class SegmentEndpoint {
     /** The center point of the endpoint */
     public final GeoPoint point;
     /** A plane describing the circle */
@@ -580,7 +580,7 @@ class GeoStandardPath extends GeoBasePath {
   /**
    * This is the pre-calculated data for a path segment.
    */
-  public static class PathSegment {
+  private static class PathSegment {
     /** Starting point of the segment */
     public final GeoPoint start;
     /** End point of the segment */


[25/50] [abbrv] lucene-solr:solr-5750: implement grow() for spatial3d intersector: easy speedup

Posted by ds...@apache.org.
implement grow() for spatial3d intersector: easy speedup


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

Branch: refs/heads/solr-5750
Commit: e3e9114921da4b208960b1da980a7c8c1ba7b1f7
Parents: 0dc32e3
Author: Robert Muir <rm...@apache.org>
Authored: Sun Apr 24 20:09:05 2016 -0400
Committer: Robert Muir <rm...@apache.org>
Committed: Sun Apr 24 20:09:05 2016 -0400

----------------------------------------------------------------------
 .../apache/lucene/spatial3d/PointInShapeIntersectVisitor.java   | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e3e91149/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/PointInShapeIntersectVisitor.java
----------------------------------------------------------------------
diff --git a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/PointInShapeIntersectVisitor.java b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/PointInShapeIntersectVisitor.java
index d4e7309..cf94c35 100644
--- a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/PointInShapeIntersectVisitor.java
+++ b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/PointInShapeIntersectVisitor.java
@@ -39,6 +39,11 @@ class PointInShapeIntersectVisitor implements IntersectVisitor {
   }
 
   @Override
+  public void grow(int count) {
+    hits.grow(count);
+  }
+
+  @Override
   public void visit(int docID) {
     hits.add(docID);
   }


[16/50] [abbrv] lucene-solr:solr-5750: LUCENE-7242: LatLonTree should build a balanced tree

Posted by ds...@apache.org.
LUCENE-7242: LatLonTree should build a balanced tree


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

Branch: refs/heads/solr-5750
Commit: 776f9ec7c8f2a3a07c5ce5229c66c2f113291ba9
Parents: bf232d7
Author: Robert Muir <rm...@apache.org>
Authored: Fri Apr 22 12:08:16 2016 -0400
Committer: Robert Muir <rm...@apache.org>
Committed: Fri Apr 22 12:09:15 2016 -0400

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  3 +-
 .../org/apache/lucene/document/LatLonTree.java  | 69 +++++++++-----------
 2 files changed, 33 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/776f9ec7/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 4b72294..848e022 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -64,8 +64,7 @@ Optimizations
   multiple polygons and holes, with memory usage independent of
   polygon complexity. (Karl Wright, Mike McCandless, Robert Muir)
 
-* LUCENE-7159, LUCENE-7222, LUCENE-7229, LUCENE-7239: Speed up LatLonPoint 
-  polygon performance. (Robert Muir)
+* LUCENE-7159: Speed up LatLonPoint polygon performance. (Robert Muir, Ryan Ernst)
 
 * LUCENE-7211: Reduce memory & GC for spatial RPT Intersects when the number of
   matching docs is small. (Jeff Wartes, David Smiley)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/776f9ec7/lucene/sandbox/src/java/org/apache/lucene/document/LatLonTree.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonTree.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonTree.java
index 8a6e6d8..f7a2927 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonTree.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonTree.java
@@ -16,17 +16,13 @@
  */
 package org.apache.lucene.document;
 
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.Random;
 
 import org.apache.lucene.geo.Polygon;
 import org.apache.lucene.index.PointValues.Relation;
 
 /**
- * 2D polygon implementation represented as a randomized interval tree of edges.
+ * 2D polygon implementation represented as a balanced interval tree of edges.
  * <p>
  * contains() and crosses() are still O(n), but for most practical polygons 
  * are much faster than brute force.
@@ -338,45 +334,44 @@ final class LatLonTree {
    * @return root node of the tree.
    */
   private static Edge createTree(double polyLats[], double polyLons[]) {
-    // edge order is deterministic and reproducible based on the double values.
-    // TODO: make a real balanced tree instead :)
-    List<Integer> list = new ArrayList<Integer>(polyLats.length - 1);
+    Edge edges[] = new Edge[polyLats.length - 1];
     for (int i = 1; i < polyLats.length; i++) {
-      list.add(i);
-    }
-    Collections.shuffle(list, new Random(Arrays.hashCode(polyLats) ^ Arrays.hashCode(polyLons)));
-    Edge root = null;
-    for (int i : list) {
       double lat1 = polyLats[i-1];
       double lon1 = polyLons[i-1];
       double lat2 = polyLats[i];
       double lon2 = polyLons[i];
-      Edge newNode = new Edge(lat1, lon1, lat2, lon2, Math.min(lat1, lat2), Math.max(lat1, lat2));
-      if (root == null) {
-        // add first node
-        root = newNode;
-      } else {
-        // traverse tree to find home for new node, along the path updating all parent's max value along the way.
-        Edge node = root;
-        while (true) {
-          node.max = Math.max(node.max, newNode.max);
-          if (newNode.low < node.low) {
-            if (node.left == null) {
-              node.left = newNode;
-              break;
-            }
-            node = node.left;
-          } else {
-            if (node.right == null) {
-              node.right = newNode;
-              break;
-            }
-            node = node.right;
-          }
-        }
+      edges[i - 1] = new Edge(lat1, lon1, lat2, lon2, Math.min(lat1, lat2), Math.max(lat1, lat2));
+    }
+    // sort the edges then build a balanced tree from them
+    Arrays.sort(edges, (left, right) -> {
+      int ret = Double.compare(left.low, right.low);
+      if (ret == 0) {
+        ret = Double.compare(left.max, right.max);
       }
+      return ret;
+    });
+    return createTree(edges, 0, edges.length - 1);
+  }
+
+  /** Creates tree from sorted edges (with range low and high inclusive) */
+  private static Edge createTree(Edge edges[], int low, int high) {
+    if (low > high) {
+      return null;
+    }
+    // add midpoint
+    int mid = (low + high) >>> 1;
+    Edge newNode = edges[mid];
+    // add children
+    newNode.left = createTree(edges, low, mid - 1);
+    newNode.right = createTree(edges, mid + 1, high);
+    // pull up max values to this node
+    if (newNode.left != null) {
+      newNode.max = Math.max(newNode.max, newNode.left.max);
+    }
+    if (newNode.right != null) {
+      newNode.max = Math.max(newNode.max, newNode.right.max);
     }
-    return root;
+    return newNode;
   }
 
   /**


[22/50] [abbrv] lucene-solr:solr-5750: LUCENE-7175: give enough heap for large dim count, bytes per dim, when writing points

Posted by ds...@apache.org.
LUCENE-7175: give enough heap for large dim count, bytes per dim, when writing 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/7acf8bab
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/7acf8bab
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/7acf8bab

Branch: refs/heads/solr-5750
Commit: 7acf8babae2401300a5844ebd10da2219c99dd40
Parents: 69be7dc
Author: Mike McCandless <mi...@apache.org>
Authored: Sun Apr 24 05:58:41 2016 -0400
Committer: Mike McCandless <mi...@apache.org>
Committed: Sun Apr 24 05:58:41 2016 -0400

----------------------------------------------------------------------
 .../core/src/test/org/apache/lucene/search/TestPointQueries.java   | 2 +-
 .../src/java/org/apache/lucene/index/RandomCodec.java              | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7acf8bab/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 bc9ef75..88d89d2 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
@@ -1153,7 +1153,7 @@ public class TestPointQueries extends LuceneTestCase {
   private static Codec getCodec() {
     if (Codec.getDefault().getName().equals("Lucene60")) {
       int maxPointsInLeafNode = TestUtil.nextInt(random(), 16, 2048);
-      double maxMBSortInHeap = 4.0 + (3*random().nextDouble());
+      double maxMBSortInHeap = 5.0 + (3*random().nextDouble());
       if (VERBOSE) {
         System.out.println("TEST: using Lucene60PointsFormat with maxPointsInLeafNode=" + maxPointsInLeafNode + " and maxMBSortInHeap=" + maxMBSortInHeap);
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7acf8bab/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 73db88b..c1c33f8 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
@@ -196,7 +196,7 @@ public class RandomCodec extends AssertingCodec {
     int lowFreqCutoff = TestUtil.nextInt(random, 2, 100);
 
     maxPointsInLeafNode = TestUtil.nextInt(random, 16, 2048);
-    maxMBSortInHeap = 4.0 + (3*random.nextDouble());
+    maxMBSortInHeap = 5.0 + (3*random.nextDouble());
     bkdSplitRandomSeed = random.nextInt();
 
     add(avoidCodecs,


[38/50] [abbrv] lucene-solr:solr-5750: SOLR-9027: Add GraphTermsQuery to limit traversal on high frequency nodes

Posted by ds...@apache.org.
SOLR-9027: Add GraphTermsQuery to limit traversal on high frequency nodes


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

Branch: refs/heads/solr-5750
Commit: d66f5515e648bdf52f3ea36ae76af72742a95336
Parents: 9c69c4c
Author: jbernste <jb...@apache.org>
Authored: Tue Apr 26 10:15:03 2016 -0400
Committer: jbernste <jb...@apache.org>
Committed: Tue Apr 26 16:33:26 2016 -0400

----------------------------------------------------------------------
 .../solr/search/GraphTermsQParserPlugin.java    | 292 +++++++++++++++++++
 .../org/apache/solr/search/QParserPlugin.java   |   1 +
 .../apache/solr/search/QueryEqualityTest.java   |   9 +
 .../search/TestGraphTermsQParserPlugin.java     | 135 +++++++++
 4 files changed, 437 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d66f5515/solr/core/src/java/org/apache/solr/search/GraphTermsQParserPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/GraphTermsQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/GraphTermsQParserPlugin.java
new file mode 100644
index 0000000..5a2fe11
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/search/GraphTermsQParserPlugin.java
@@ -0,0 +1,292 @@
+
+/*
+ * 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;
+
+import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.TermContext;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermState;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.BulkScorer;
+import org.apache.lucene.search.ConstantScoreQuery;
+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.MatchNoDocsQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.DocIdSetBuilder;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.schema.FieldType;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+/**
+ *  The GraphTermsQuery builds a disjunction query from a list of terms. The terms are first filtered by the maxDocFreq parameter.
+ *  This allows graph traversals to skip traversing high frequency nodes which is often desirable from a performance standpoint.
+ *
+ *   Syntax: {!graphTerms f=field maxDocFreq=10000}term1,term2,term3
+ **/
+
+public class GraphTermsQParserPlugin extends QParserPlugin {
+  public static final String NAME = "graphTerms";
+
+  @Override
+  public QParser createParser(String qstr, SolrParams localParams, SolrParams params, SolrQueryRequest req) {
+    return new QParser(qstr, localParams, params, req) {
+      @Override
+      public Query parse() throws SyntaxError {
+        String fname = localParams.get(QueryParsing.F);
+        FieldType ft = req.getSchema().getFieldTypeNoEx(fname);
+        int maxDocFreq = localParams.getInt("maxDocFreq", Integer.MAX_VALUE);
+        String qstr = localParams.get(QueryParsing.V);//never null
+
+        if (qstr.length() == 0) {
+          return new MatchNoDocsQuery();
+        }
+
+        final String[] splitVals = qstr.split(",");
+
+        BytesRef[] bytesRefs = new BytesRef[splitVals.length];
+        BytesRefBuilder term = new BytesRefBuilder();
+        for (int i = 0; i < splitVals.length; i++) {
+          String stringVal = splitVals[i].trim();
+          if (ft != null) {
+            ft.readableToIndexed(stringVal, term);
+          } else {
+            term.copyChars(stringVal);
+          }
+          bytesRefs[i] = term.toBytesRef();
+        }
+
+        return new ConstantScoreQuery(new GraphTermsQuery(fname, bytesRefs, maxDocFreq));
+      }
+    };
+  }
+
+  private class GraphTermsQuery extends Query {
+
+    private Term[] queryTerms;
+    private List<TermContext> finalContexts;
+    private List<Term> finalTerms;
+    private String field;
+    private int maxDocFreq;
+    private Object id;
+
+    public GraphTermsQuery(String field, BytesRef[] terms, int maxDocFreq) {
+      this.maxDocFreq = maxDocFreq;
+      this.field = field;
+      this.queryTerms = new Term[terms.length];
+      this.id = new Object();
+      for(int i=0; i<terms.length; i++) {
+        this.queryTerms[i] = new Term(field, terms[i]);
+      }
+    }
+
+    //Just for cloning
+    private GraphTermsQuery(String field, Term[] terms, int maxDocFreq, Object id) {
+      this.field = field;
+      this.queryTerms = terms;
+      this.maxDocFreq = maxDocFreq;
+      this.id = id;
+    }
+
+    @Override
+    public Query rewrite(IndexReader reader) throws IOException {
+      if(this.finalContexts == null) {
+        //This query has not been re-written yet.
+        //Rewriting the query does not effect the cache key as this query is not designed to be cached.
+        this.finalContexts = new ArrayList();
+        this.finalTerms = new ArrayList();
+        List<LeafReaderContext> contexts = reader.leaves();
+        TermContext[] termContexts = new TermContext[this.queryTerms.length];
+        collectTermContext(reader, contexts, termContexts, this.queryTerms);
+        for(int i=0; i<termContexts.length; i++) {
+          TermContext termContext = termContexts[i];
+          if(termContext != null && termContext.docFreq() < this.maxDocFreq) {
+            this.finalContexts.add(termContext);
+            this.finalTerms.add(queryTerms[i]);
+          }
+        }
+      }
+
+      return this;
+    }
+
+    public int hashCode() {
+      return 31 * super.hashCode() + id.hashCode();
+    }
+
+    public boolean equals(Object o) {
+      if (super.equals(o) == false) {
+        return false;
+      }
+
+      GraphTermsQuery q = (GraphTermsQuery)o;
+      return id == q.id;
+    }
+
+    public GraphTermsQuery clone() {
+      GraphTermsQuery clone = new GraphTermsQuery(this.field, this.queryTerms, this.maxDocFreq, this.id);
+      return clone;
+    }
+
+    @Override
+    public String toString(String defaultField) {
+      StringBuilder builder = new StringBuilder();
+      boolean first = true;
+      for (Term term : this.queryTerms) {
+        if (!first) {
+          builder.append(',');
+        }
+        first = false;
+        builder.append(term.toString());
+      }
+
+      return builder.toString();
+    }
+
+    private class WeightOrDocIdSet {
+      final Weight weight;
+      final DocIdSet set;
+
+      WeightOrDocIdSet(DocIdSet bitset) {
+        this.set = bitset;
+        this.weight = null;
+      }
+    }
+
+    @Override
+    public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
+      return new ConstantScoreWeight(this) {
+
+        @Override
+        public void extractTerms(Set<Term> terms) {
+          // no-op
+          // This query is for abuse cases when the number of terms is too high to
+          // run efficiently as a BooleanQuery. So likewise we hide its terms in
+          // order to protect highlighters
+        }
+
+        private WeightOrDocIdSet rewrite(LeafReaderContext context) throws IOException {
+          final LeafReader reader = context.reader();
+          final Fields fields = reader.fields();
+          Terms terms = fields.terms(field);
+          TermsEnum  termsEnum = terms.iterator();
+          PostingsEnum docs = null;
+          DocIdSetBuilder builder = new DocIdSetBuilder(reader.maxDoc());
+          for (int i=0; i<finalContexts.size(); i++) {
+            TermContext termContext = finalContexts.get(i);
+            TermState termState = termContext.get(context.ord);
+            if(termState != null) {
+              Term term = finalTerms.get(i);
+              termsEnum.seekExact(term.bytes(), termContext.get(context.ord));
+              docs = termsEnum.postings(docs, PostingsEnum.NONE);
+              builder.add(docs);
+            }
+          }
+          return new WeightOrDocIdSet(builder.build());
+        }
+
+        private Scorer scorer(DocIdSet set) throws IOException {
+          if (set == null) {
+            return null;
+          }
+          final DocIdSetIterator disi = set.iterator();
+          if (disi == null) {
+            return null;
+          }
+          return new ConstantScoreScorer(this, score(), disi);
+        }
+
+        @Override
+        public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
+          final WeightOrDocIdSet weightOrBitSet = rewrite(context);
+          if (weightOrBitSet.weight != null) {
+            return weightOrBitSet.weight.bulkScorer(context);
+          } else {
+            final Scorer scorer = scorer(weightOrBitSet.set);
+            if (scorer == null) {
+              return null;
+            }
+            return new DefaultBulkScorer(scorer);
+          }
+        }
+
+        @Override
+        public Scorer scorer(LeafReaderContext context) throws IOException {
+          final WeightOrDocIdSet weightOrBitSet = rewrite(context);
+          if (weightOrBitSet.weight != null) {
+            return weightOrBitSet.weight.scorer(context);
+          } else {
+            return scorer(weightOrBitSet.set);
+          }
+        }
+      };
+    }
+
+    private void collectTermContext(IndexReader reader,
+                                    List<LeafReaderContext> leaves,
+                                    TermContext[] contextArray,
+                                    Term[] queryTerms) throws IOException {
+      TermsEnum termsEnum = null;
+      for (LeafReaderContext context : leaves) {
+        final Fields fields = context.reader().fields();
+        for (int i = 0; i < queryTerms.length; i++) {
+          Term term = queryTerms[i];
+          TermContext termContext = contextArray[i];
+          final Terms terms = fields.terms(term.field());
+          if (terms == null) {
+            // field does not exist
+            continue;
+          }
+          termsEnum = terms.iterator();
+          assert termsEnum != null;
+
+          if (termsEnum == TermsEnum.EMPTY) continue;
+          if (termsEnum.seekExact(term.bytes())) {
+            if (termContext == null) {
+              contextArray[i] = new TermContext(reader.getContext(),
+                  termsEnum.termState(), context.ord, termsEnum.docFreq(),
+                  termsEnum.totalTermFreq());
+            } else {
+              termContext.register(termsEnum.termState(), context.ord,
+                  termsEnum.docFreq(), termsEnum.totalTermFreq());
+            }
+
+          }
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d66f5515/solr/core/src/java/org/apache/solr/search/QParserPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/QParserPlugin.java b/solr/core/src/java/org/apache/solr/search/QParserPlugin.java
index cf556ff..7a6247f 100644
--- a/solr/core/src/java/org/apache/solr/search/QParserPlugin.java
+++ b/solr/core/src/java/org/apache/solr/search/QParserPlugin.java
@@ -76,6 +76,7 @@ public abstract class QParserPlugin implements NamedListInitializedPlugin, SolrI
     map.put(HashQParserPlugin.NAME, HashQParserPlugin.class);
     map.put(GraphQParserPlugin.NAME, GraphQParserPlugin.class);
     map.put(XmlQParserPlugin.NAME, XmlQParserPlugin.class);
+    map.put(GraphTermsQParserPlugin.NAME, GraphTermsQParserPlugin.class);
     standardPlugins = Collections.unmodifiableMap(map);
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d66f5515/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 a9b0209..2a89473 100644
--- a/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java
+++ b/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java
@@ -165,6 +165,15 @@ public class QueryEqualityTest extends SolrTestCaseJ4 {
     }
   }
 
+  public void testGraphTermsQuery() throws Exception {
+    SolrQueryRequest req = req("q", "*:*");
+    try {
+      assertQueryEquals("graphTerms", req, "{!graphTerms f=field1 maxDocFreq=1000}term1,term2");
+    } finally {
+      req.close();
+    }
+  }
+
   public void testQuerySwitch() throws Exception {
     SolrQueryRequest req = req("myXXX", "XXX", 
                                "myField", "foo_s",

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d66f5515/solr/core/src/test/org/apache/solr/search/TestGraphTermsQParserPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestGraphTermsQParserPlugin.java b/solr/core/src/test/org/apache/solr/search/TestGraphTermsQParserPlugin.java
new file mode 100644
index 0000000..cfff980
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/search/TestGraphTermsQParserPlugin.java
@@ -0,0 +1,135 @@
+package org.apache.solr.search;
+
+/*
+ * 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.
+ */
+
+import java.util.Collections;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.Iterator;
+
+import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.search.CollapsingQParserPlugin.GroupHeadSelector;
+import org.apache.solr.search.CollapsingQParserPlugin.GroupHeadSelectorType;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+//We want codecs that support DocValues, and ones supporting blank/empty values.
+@SuppressCodecs({"Appending","Lucene3x","Lucene40","Lucene41","Lucene42"})
+public class TestGraphTermsQParserPlugin extends SolrTestCaseJ4 {
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    initCore("solrconfig-collapseqparser.xml", "schema11.xml");
+  }
+
+  @Override
+  @Before
+  public void setUp() throws Exception {
+    // if you override setUp or tearDown, you better call
+    // the super classes version
+    super.setUp();
+    clearIndex();
+    assertU(commit());
+  }
+
+  @Test
+  public void testQueries() throws Exception {
+
+    String group = "group_s";
+
+    String[] doc = {"id","1", "term_s", "YYYY", group, "1", "test_ti", "5", "test_tl", "10", "test_tf", "2000"};
+    assertU(adoc(doc));
+    String[] doc1 = {"id","2", "term_s","YYYY", group, "1", "test_ti", "5", "test_tl", "100", "test_tf", "200"};
+    assertU(adoc(doc1));
+
+    String[] doc2 = {"id","3", "term_s", "YYYY", "test_ti", "5000", "test_tl", "100", "test_tf", "200"};
+    assertU(adoc(doc2));
+    assertU(commit());
+    String[] doc3 = {"id","4", "term_s", "YYYY", "test_ti", "500", "test_tl", "1000", "test_tf", "2000"};
+    assertU(adoc(doc3));
+
+    String[] doc4 = {"id","5", "term_s", "YYYY", group, "2", "test_ti", "5", "test_tl", "10", "test_tf", "2000"};
+    assertU(adoc(doc4));
+    assertU(commit());
+    String[] doc5 = {"id","6", "term_s","YYYY", group, "2", "test_ti", "10", "test_tl", "100", "test_tf", "200"};
+    assertU(adoc(doc5));
+    assertU(commit());
+
+    String[] doc6 = {"id","7", "term_s", "YYYY", group, "1", "test_ti", "10", "test_tl", "50", "test_tf", "300"};
+    assertU(adoc(doc6));
+    assertU(commit());
+
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.add("q", "{!graphTerms f=group_s maxDocFreq=10}1,2");
+    params.add("sort", "id asc");
+    assertQ(req(params, "indent", "on"), "*[count(//doc)=5]",
+        "//result/doc[1]/float[@name='id'][.='1.0']",
+        "//result/doc[2]/float[@name='id'][.='2.0']",
+        "//result/doc[3]/float[@name='id'][.='5.0']",
+        "//result/doc[4]/float[@name='id'][.='6.0']",
+        "//result/doc[5]/float[@name='id'][.='7.0']"
+    );
+
+    //Test without maxDocFreq param. Should default to Integer.MAX_VALUE and match all terms.
+    params = new ModifiableSolrParams();
+    params.add("q", "{!graphTerms f=group_s}1,2");
+    params.add("sort", "id asc");
+    assertQ(req(params, "indent", "on"), "*[count(//doc)=5]",
+        "//result/doc[1]/float[@name='id'][.='1.0']",
+        "//result/doc[2]/float[@name='id'][.='2.0']",
+        "//result/doc[3]/float[@name='id'][.='5.0']",
+        "//result/doc[4]/float[@name='id'][.='6.0']",
+        "//result/doc[5]/float[@name='id'][.='7.0']"
+    );
+
+    params = new ModifiableSolrParams();
+    params.add("q", "{!graphTerms f=group_s maxDocFreq=1}1,2");
+    params.add("sort", "id asc");
+    assertQ(req(params, "indent", "on"), "*[count(//doc)=0]"
+    );
+
+    //Test with int field
+    params = new ModifiableSolrParams();
+    params.add("q", "{!graphTerms f=test_ti maxDocFreq=10}5,10");
+    params.add("sort", "id asc");
+    assertQ(req(params, "indent", "on"), "*[count(//doc)=5]",
+        "//result/doc[1]/float[@name='id'][.='1.0']",
+        "//result/doc[2]/float[@name='id'][.='2.0']",
+        "//result/doc[3]/float[@name='id'][.='5.0']",
+        "//result/doc[4]/float[@name='id'][.='6.0']",
+        "//result/doc[5]/float[@name='id'][.='7.0']"
+    );
+
+    //Test with int field
+    params = new ModifiableSolrParams();
+    params.add("q", "{!graphTerms f=test_ti maxDocFreq=3}5,10");
+    params.add("sort", "id asc");
+    assertQ(req(params, "indent", "on"), "*[count(//doc)=2]",
+        "//result/doc[1]/float[@name='id'][.='6.0']",
+        "//result/doc[2]/float[@name='id'][.='7.0']"
+    );
+  }
+}


[12/50] [abbrv] lucene-solr:solr-5750: SOLR-8992: Restore Schema API GET method functionality removed in 6.0 added back another test and fixed test errors

Posted by ds...@apache.org.
SOLR-8992: Restore Schema API GET method functionality removed in 6.0 added back another test and fixed test errors


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

Branch: refs/heads/solr-5750
Commit: 827c670d189f9da56259b541db9b94113d3ca9a0
Parents: dac6fe0
Author: Noble Paul <no...@apache.org>
Authored: Fri Apr 22 11:32:29 2016 +0530
Committer: Noble Paul <no...@apache.org>
Committed: Fri Apr 22 11:32:29 2016 +0530

----------------------------------------------------------------------
 .../org/apache/solr/handler/SchemaHandler.java  |  44 ++++----
 .../org/apache/solr/schema/IndexSchema.java     | 105 +++++++++++--------
 .../solr/rest/schema/TestFieldResource.java     |   6 ++
 .../solrj/response/schema/SchemaResponse.java   |  64 +++++------
 .../org/apache/solr/common/util/NamedList.java  |  69 ++++++++++++
 5 files changed, 194 insertions(+), 94 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/827c670d/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java b/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java
index 4da557d..063c3d4 100644
--- a/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java
@@ -20,20 +20,19 @@ import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.Arrays;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
-import java.util.Locale;
 import java.util.Map;
 import java.util.Set;
 
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
 import org.apache.solr.cloud.ZkSolrResourceLoader;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.params.MapSolrParams;
+import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.ContentStream;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestHandler;
@@ -48,7 +47,12 @@ import org.apache.solr.util.plugin.SolrCoreAware;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static java.util.Collections.singletonMap;
 import static org.apache.solr.common.params.CommonParams.JSON;
+import static org.apache.solr.schema.IndexSchema.SchemaProps.Handler.COPY_FIELDS;
+import static org.apache.solr.schema.IndexSchema.SchemaProps.Handler.DYNAMIC_FIELDS;
+import static org.apache.solr.schema.IndexSchema.SchemaProps.Handler.FIELDS;
+import static org.apache.solr.schema.IndexSchema.SchemaProps.Handler.FIELD_TYPES;
 
 public class SchemaHandler extends RequestHandlerBase implements SolrCoreAware, PermissionNameProvider {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@@ -57,18 +61,14 @@ public class SchemaHandler extends RequestHandlerBase implements SolrCoreAware,
   private static final Map<String, String> level2;
 
   static {
-    Set<String> s = ImmutableSet.of(
-        IndexSchema.FIELD_TYPES,
-        IndexSchema.FIELDS,
-        IndexSchema.DYNAMIC_FIELDS,
-        IndexSchema.COPY_FIELDS
+    Map s = Utils.makeMap(
+        FIELD_TYPES.nameLower, null,
+        FIELDS.nameLower, "fl",
+        DYNAMIC_FIELDS.nameLower, "fl",
+        COPY_FIELDS.nameLower, null
     );
-    Map<String, String> m = new HashMap<>();
-    for (String s1 : s) {
-      m.put(s1, s1);
-      m.put(s1.toLowerCase(Locale.ROOT), s1);
-    }
-    level2 = ImmutableMap.copyOf(m);
+
+    level2 = Collections.unmodifiableMap(s);
   }
 
 
@@ -184,9 +184,15 @@ public class SchemaHandler extends RequestHandlerBase implements SolrCoreAware,
           List<String> parts = StrUtils.splitSmart(path, '/');
           if (parts.get(0).isEmpty()) parts.remove(0);
           if (parts.size() > 1 && level2.containsKey(parts.get(1))) {
-            String realName = level2.get(parts.get(1));
+            String realName = parts.get(1);
+            String fieldName = IndexSchema.SchemaProps.nameMapping.get(realName);
+
+            String pathParam = level2.get(realName);
+            if (parts.size() > 2) {
+              req.setParams(SolrParams.wrapDefaults(new MapSolrParams(singletonMap(pathParam, parts.get(2))), req.getParams()));
+            }
             Map propertyValues = req.getSchema().getNamedPropertyValues(realName, req.getParams());
-            Object o = propertyValues.get(realName);
+            Object o = propertyValues.get(fieldName);
             if(parts.size()> 2) {
               String name = parts.get(2);
               if (o instanceof List) {
@@ -195,7 +201,7 @@ public class SchemaHandler extends RequestHandlerBase implements SolrCoreAware,
                   if (obj instanceof SimpleOrderedMap) {
                     SimpleOrderedMap simpleOrderedMap = (SimpleOrderedMap) obj;
                     if(name.equals(simpleOrderedMap.get("name"))) {
-                      rsp.add(realName.substring(0, realName.length() - 1), simpleOrderedMap);
+                      rsp.add(fieldName.substring(0, realName.length() - 1), simpleOrderedMap);
                       return;
                     }
                   }
@@ -203,7 +209,7 @@ public class SchemaHandler extends RequestHandlerBase implements SolrCoreAware,
               }
               throw new SolrException(SolrException.ErrorCode.NOT_FOUND, "No such path " + path);
             } else {
-              rsp.add(realName, o);
+              rsp.add(fieldName, o);
             }
             return;
           }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/827c670d/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/IndexSchema.java b/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
index f71db0a..9e65332 100644
--- a/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
+++ b/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
@@ -16,19 +16,32 @@
  */
 package org.apache.solr.schema;
 
+import javax.xml.xpath.XPath;
+import javax.xml.xpath.XPathConstants;
+import javax.xml.xpath.XPathExpressionException;
 import java.io.IOException;
 import java.io.Writer;
 import java.lang.invoke.MethodHandles;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.TreeSet;
 import java.util.function.Function;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
-import javax.xml.xpath.XPath;
-import javax.xml.xpath.XPathConstants;
-import javax.xml.xpath.XPathExpressionException;
 
-import com.google.common.base.Functions;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.DelegatingAnalyzerWrapper;
 import org.apache.lucene.index.DocValuesType;
@@ -40,8 +53,8 @@ import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.uninverting.UninvertingReader;
 import org.apache.lucene.util.Version;
-import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.MapSolrParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
@@ -1363,10 +1376,10 @@ public class IndexSchema {
     return getNamedPropertyValues(null, new MapSolrParams(Collections.EMPTY_MAP));
   }
 
-  static class SchemaProps implements MapSerializable {
+  public static class SchemaProps implements MapSerializable {
     private static final String SOURCE_FIELD_LIST = IndexSchema.SOURCE + "." + CommonParams.FL;
     private static final String DESTINATION_FIELD_LIST = IndexSchema.DESTINATION + "." + CommonParams.FL;
-    private final String name;
+    public final String name;
     private final SolrParams params;
     private final IndexSchema schema;
     boolean showDefaults, includeDynamic;
@@ -1375,7 +1388,7 @@ public class IndexSchema {
     private Set<String> requestedDestinationFields;
 
 
-    enum Handler {
+    public enum Handler {
       NAME(IndexSchema.NAME, sp -> sp.schema.getSchemaName()),
       VERSION(IndexSchema.VERSION, sp -> sp.schema.getVersion()),
       UNIQUE_KEY(IndexSchema.UNIQUE_KEY, sp -> sp.schema.uniqueKeyFieldName),
@@ -1391,11 +1404,6 @@ public class IndexSchema {
           .map(it -> it.getNamedPropertyValues(sp.showDefaults))
           .collect(Collectors.toList())),
 
-      DYNAMIC_FIELDS(IndexSchema.DYNAMIC_FIELDS, sp -> Stream.of(sp.schema.dynamicFields)
-          .filter(it -> !it.getRegex().startsWith(INTERNAL_POLY_FIELD_PREFIX))
-          .filter(it -> sp.requestedFields == null || sp.requestedFields.contains(it.getPrototype().getName()))
-          .map(it -> sp.getProperties(it.getPrototype()))
-          .collect(Collectors.toList())),
       FIELDS(IndexSchema.FIELDS, sp -> {
         List<SimpleOrderedMap> result = (sp.requestedFields != null ? sp.requestedFields : new TreeSet<>(sp.schema.fields.keySet()))
             .stream()
@@ -1405,57 +1413,65 @@ public class IndexSchema {
             .map(sp::getProperties)
             .collect(Collectors.toList());
         if (sp.includeDynamic && sp.requestedFields == null) {
-          result.addAll((Collection) Handler.DYNAMIC_FIELDS.fun.apply(sp));
+          result.addAll(sp.applyDynamic());
         }
         return result;
       }),
+      DYNAMIC_FIELDS(IndexSchema.DYNAMIC_FIELDS, sp -> Stream.of(sp.schema.dynamicFields)
+          .filter(it -> !it.getRegex().startsWith(INTERNAL_POLY_FIELD_PREFIX))
+          .filter(it -> sp.requestedFields == null || sp.requestedFields.contains(it.getPrototype().getName()))
+          .map(it -> sp.getProperties(it.getPrototype()))
+          .collect(Collectors.toList())),
 
 
       COPY_FIELDS(IndexSchema.COPY_FIELDS, sp -> sp.schema.getCopyFieldProperties(false,
           sp.requestedSourceFields, sp.requestedDestinationFields));
 
       final Function<SchemaProps, Object> fun;
-      final String name;
+      public final String realName, nameLower;
       Handler(String name, Function<SchemaProps, Object> fun) {
         this.fun = fun;
-        this.name = name;
+        this.realName = name;
+        nameLower = name.toLowerCase(Locale.ROOT);
+
+      }
+      public String getRealName(){
+        return realName;
+      }
+      public String getNameLower(){
+        return nameLower;
+
       }
     }
 
+
     SchemaProps(String name, SolrParams params, IndexSchema schema) {
       this.name = name;
       this.params = params;
       this.schema = schema;
       showDefaults = params.getBool("showDefaults", false);
       includeDynamic = params.getBool("includeDynamic", false);
+      requestedSourceFields = readMultiVals(SOURCE_FIELD_LIST);
+      requestedDestinationFields = readMultiVals(DESTINATION_FIELD_LIST);
+      requestedFields = readMultiVals(CommonParams.FL);
 
-      String sourceFieldListParam = params.get(SOURCE_FIELD_LIST);
-      if (null != sourceFieldListParam) {
-        String[] fields = sourceFieldListParam.trim().split("[,\\s]+");
-        if (fields.length > 0) {
-          requestedSourceFields = new HashSet<>(Arrays.asList(fields));
-          requestedSourceFields.remove(""); // Remove empty values, if any
-        }
-      }
-      String destinationFieldListParam = params.get(DESTINATION_FIELD_LIST);
-      if (null != destinationFieldListParam) {
-        String[] fields = destinationFieldListParam.trim().split("[,\\s]+");
-        if (fields.length > 0) {
-          requestedDestinationFields = new HashSet<>(Arrays.asList(fields));
-          requestedDestinationFields.remove(""); // Remove empty values, if any
-        }
-      }
+    }
+    public Collection applyDynamic(){
+      return (Collection) Handler.DYNAMIC_FIELDS.fun.apply(this);
+    }
 
-      String flParam = params.get(CommonParams.FL);
+    private Set<String> readMultiVals(String name) {
+      String flParam = params.get(name);
       if (null != flParam) {
         String[] fields = flParam.trim().split("[,\\s]+");
         if (fields.length > 0)
-          requestedFields = new LinkedHashSet<>(Stream.of(fields)
+          return new LinkedHashSet<>(Stream.of(fields)
               .filter(it -> !it.trim().isEmpty())
               .collect(Collectors.toList()));
 
       }
 
+      return null;
     }
 
 
@@ -1474,15 +1490,18 @@ public class IndexSchema {
 
     @Override
     public Map<String, Object> toMap() {
-      Map<String, Object> topLevel = new LinkedHashMap<>();
-      Stream.of(Handler.values())
-          .filter(it -> name == null || it.name.equals(name))
-          .forEach(it -> {
-            Object val = it.fun.apply(this);
-            if (val != null) topLevel.put(it.name, val);
-          });
-      return topLevel;
+      return Stream.of(Handler.values())
+          .filter(it -> name == null || it.nameLower.equals(name))
+          .map(it -> new Pair<>(it.realName, it.fun.apply(this)))
+          .filter(it->it.getValue() != null)
+          .collect(Collectors.toMap(
+              Pair::getKey,
+              Pair::getValue,
+              (v1, v2) -> v2,
+              LinkedHashMap::new));
     }
+    public static Map<String,String> nameMapping = Collections.unmodifiableMap(Stream.of(Handler.values())
+        .collect(Collectors.toMap(Handler::getNameLower , Handler::getRealName)));
   }
 
   public Map<String, Object> getNamedPropertyValues(String name, SolrParams params) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/827c670d/solr/core/src/test/org/apache/solr/rest/schema/TestFieldResource.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/rest/schema/TestFieldResource.java b/solr/core/src/test/org/apache/solr/rest/schema/TestFieldResource.java
index 627aee0..d591b9a 100644
--- a/solr/core/src/test/org/apache/solr/rest/schema/TestFieldResource.java
+++ b/solr/core/src/test/org/apache/solr/rest/schema/TestFieldResource.java
@@ -71,6 +71,12 @@ public class TestFieldResource extends SolrRestletTestBase {
              "/field/required==false",
              "/field/tokenized==true");
   }
+  @Test
+  public void testGetFieldIncludeDynamic() throws Exception {
+    assertQ("/schema/fields/some_crazy_name_i?indent=on&wt=xml&includeDynamic=true",
+        "/response/lst[@name='field']/str[@name='name'] = 'some_crazy_name_i'",
+        "/response/lst[@name='field']/str[@name='dynamicBase'] = '*_i'");
+  }
   
 
   @Test

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/827c670d/solr/solrj/src/java/org/apache/solr/client/solrj/response/schema/SchemaResponse.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/response/schema/SchemaResponse.java b/solr/solrj/src/java/org/apache/solr/client/solrj/response/schema/SchemaResponse.java
index 0f3b289..088cd60 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/response/schema/SchemaResponse.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/response/schema/SchemaResponse.java
@@ -132,38 +132,38 @@ public class SchemaResponse extends SolrResponseBase {
     }
   }
 
-  private static SchemaRepresentation createSchemaConfiguration(NamedList<Object> schemaNamedList) {
+  private static SchemaRepresentation createSchemaConfiguration(Map schemaObj) {
     SchemaRepresentation schemaRepresentation = new SchemaRepresentation();
-    schemaRepresentation.setName(getSchemaName(schemaNamedList));
-    schemaRepresentation.setVersion(getSchemaVersion(schemaNamedList));
-    schemaRepresentation.setUniqueKey(getSchemaUniqueKey(schemaNamedList));
-    schemaRepresentation.setDefaultSearchField(getDefaultSearchField(schemaNamedList));
-    schemaRepresentation.setDefaultOperator(getDefaultOperator(schemaNamedList));
-    schemaRepresentation.setSimilarity(getSimilarity(schemaNamedList));
-    schemaRepresentation.setFields(getFields(schemaNamedList));
-    schemaRepresentation.setDynamicFields(getDynamicFields(schemaNamedList));
-    schemaRepresentation.setFieldTypes(getFieldTypeDefinitions(schemaNamedList));
-    schemaRepresentation.setCopyFields(getCopyFields(schemaNamedList));
+    schemaRepresentation.setName(getSchemaName(schemaObj));
+    schemaRepresentation.setVersion(getSchemaVersion(schemaObj));
+    schemaRepresentation.setUniqueKey(getSchemaUniqueKey(schemaObj));
+    schemaRepresentation.setDefaultSearchField(getDefaultSearchField(schemaObj));
+    schemaRepresentation.setDefaultOperator(getDefaultOperator(schemaObj));
+    schemaRepresentation.setSimilarity(getSimilarity(schemaObj));
+    schemaRepresentation.setFields(getFields(schemaObj));
+    schemaRepresentation.setDynamicFields(getDynamicFields(schemaObj));
+    schemaRepresentation.setFieldTypes(getFieldTypeDefinitions(schemaObj));
+    schemaRepresentation.setCopyFields(getCopyFields(schemaObj));
     return schemaRepresentation;
   }
 
-  private static String getSchemaName(NamedList<Object> schemaNamedList) {
+  private static String getSchemaName(Map schemaNamedList) {
     return (String) schemaNamedList.get("name");
   }
 
-  private static Float getSchemaVersion(NamedList<Object> schemaNamedList) {
+  private static Float getSchemaVersion(Map schemaNamedList) {
     return (Float) schemaNamedList.get("version");
   }
 
-  private static String getSchemaUniqueKey(NamedList<Object> schemaNamedList) {
+  private static String getSchemaUniqueKey(Map schemaNamedList) {
     return (String) schemaNamedList.get("uniqueKey");
   }
 
-  private static String getDefaultSearchField(NamedList<Object> schemaNamedList) {
+  private static String getDefaultSearchField(Map schemaNamedList) {
     return (String) schemaNamedList.get("defaultSearchField");
   }
 
-  private static Map<String, Object> getSimilarity(NamedList<Object> schemaNamedList) {
+  private static Map<String, Object> getSimilarity(Map schemaNamedList) {
     NamedList<Object> similarityNamedList = (NamedList<Object>) schemaNamedList.get("similarity");
     Map<String, Object> similarity = null;
     if (similarityNamedList != null) similarity = extractAttributeMap(similarityNamedList);
@@ -171,7 +171,7 @@ public class SchemaResponse extends SolrResponseBase {
   }
 
   @SuppressWarnings("unchecked")
-  private static String getDefaultOperator(NamedList<Object> schemaNamedList) {
+  private static String getDefaultOperator(Map schemaNamedList) {
     String defaultOperator = null;
     NamedList<Object> solrQueryParserProperties = (NamedList<Object>) schemaNamedList.get("solrQueryParser");
     if (solrQueryParserProperties != null) defaultOperator = (String) solrQueryParserProperties.get("defaultOperator");
@@ -179,7 +179,7 @@ public class SchemaResponse extends SolrResponseBase {
   }
 
   @SuppressWarnings("unchecked")
-  private static List<Map<String, Object>> getFields(NamedList<Object> schemaNamedList) {
+  private static List<Map<String, Object>> getFields(Map schemaNamedList) {
     List<Map<String, Object>> fieldsAttributes = new LinkedList<>();
     List<NamedList<Object>> fieldsResponse = (List<NamedList<Object>>) schemaNamedList.get("fields");
     for (NamedList<Object> fieldNamedList : fieldsResponse) {
@@ -192,7 +192,7 @@ public class SchemaResponse extends SolrResponseBase {
   }
 
   @SuppressWarnings("unchecked")
-  private static List<Map<String, Object>> getDynamicFields(NamedList<Object> schemaNamedList) {
+  private static List<Map<String, Object>> getDynamicFields(Map schemaNamedList) {
     List<Map<String, Object>> dynamicFieldsAttributes = new LinkedList<>();
     List<NamedList<Object>> dynamicFieldsResponse = (List<NamedList<Object>>) schemaNamedList.get("dynamicFields");
     for (NamedList<Object> fieldNamedList : dynamicFieldsResponse) {
@@ -205,7 +205,7 @@ public class SchemaResponse extends SolrResponseBase {
   }
 
   @SuppressWarnings("unchecked")
-  private static List<Map<String, Object>> getCopyFields(NamedList<Object> schemaNamedList) {
+  private static List<Map<String, Object>> getCopyFields(Map schemaNamedList) {
     List<Map<String, Object>> copyFieldsAttributes = new LinkedList<>();
     List<NamedList<Object>> copyFieldsResponse = (List<NamedList<Object>>) schemaNamedList.get("copyFields");
     for (NamedList<Object> copyFieldNamedList : copyFieldsResponse) {
@@ -218,7 +218,7 @@ public class SchemaResponse extends SolrResponseBase {
   }
 
   @SuppressWarnings("unchecked")
-  private static List<FieldTypeDefinition> getFieldTypeDefinitions(NamedList<Object> schemaNamedList) {
+  private static List<FieldTypeDefinition> getFieldTypeDefinitions(Map schemaNamedList) {
     List<FieldTypeDefinition> fieldTypeDefinitions = new LinkedList<>();
     List<NamedList<Object>> fieldsResponse = (List<NamedList<Object>>) schemaNamedList.get("fieldTypes");
     for (NamedList<Object> fieldNamedList : fieldsResponse) {
@@ -230,7 +230,7 @@ public class SchemaResponse extends SolrResponseBase {
   }
 
   @SuppressWarnings("unchecked")
-  private static List<FieldTypeRepresentation> getFieldTypeRepresentations(NamedList<Object> schemaNamedList) {
+  private static List<FieldTypeRepresentation> getFieldTypeRepresentations(Map schemaNamedList) {
     List<FieldTypeRepresentation> fieldTypeRepresentations = new LinkedList<>();
     List<NamedList<Object>> fieldsResponse = (List<NamedList<Object>>) schemaNamedList.get("fieldTypes");
     for (NamedList<Object> fieldNamedList : fieldsResponse) {
@@ -249,8 +249,8 @@ public class SchemaResponse extends SolrResponseBase {
   public void setResponse(NamedList<Object> response) {
     super.setResponse(response);
 
-    NamedList<Object> schemaNamedList = (NamedList<Object>) response.get("schema");
-    schemaRepresentation = createSchemaConfiguration(schemaNamedList);
+    Map schemaObj = (Map) response.get("schema");
+    schemaRepresentation = createSchemaConfiguration(schemaObj);
   }
 
   public SchemaRepresentation getSchemaRepresentation() {
@@ -268,7 +268,7 @@ public class SchemaResponse extends SolrResponseBase {
     public void setResponse(NamedList<Object> response) {
       super.setResponse(response);
 
-      schemaName = SchemaResponse.getSchemaName(response);
+      schemaName = SchemaResponse.getSchemaName(response.asShallowMap());
     }
 
     public String getSchemaName() {
@@ -288,7 +288,7 @@ public class SchemaResponse extends SolrResponseBase {
     public void setResponse(NamedList<Object> response) {
       super.setResponse(response);
 
-      schemaVersion = SchemaResponse.getSchemaVersion(response);
+      schemaVersion = SchemaResponse.getSchemaVersion(response.asShallowMap());
     }
 
     public float getSchemaVersion() {
@@ -329,7 +329,7 @@ public class SchemaResponse extends SolrResponseBase {
     public void setResponse(NamedList<Object> response) {
       super.setResponse(response);
 
-      fields = SchemaResponse.getFields(response);
+      fields = SchemaResponse.getFields(response.asShallowMap());
     }
 
     public List<Map<String, Object>> getFields() {
@@ -369,7 +369,7 @@ public class SchemaResponse extends SolrResponseBase {
     public void setResponse(NamedList<Object> response) {
       super.setResponse(response);
 
-      dynamicFields = SchemaResponse.getDynamicFields(response);
+      dynamicFields = SchemaResponse.getDynamicFields(response.asMap(3));
     }
 
     public List<Map<String, Object>> getDynamicFields() {
@@ -388,7 +388,7 @@ public class SchemaResponse extends SolrResponseBase {
     public void setResponse(NamedList<Object> response) {
       super.setResponse(response);
 
-      uniqueKey = SchemaResponse.getSchemaUniqueKey(response);
+      uniqueKey = SchemaResponse.getSchemaUniqueKey(response.asShallowMap());
     }
 
     public String getUniqueKey() {
@@ -407,7 +407,7 @@ public class SchemaResponse extends SolrResponseBase {
     public void setResponse(NamedList<Object> response) {
       super.setResponse(response);
 
-      similarity = SchemaResponse.getSimilarity(response);
+      similarity = SchemaResponse.getSimilarity(response.asShallowMap());
     }
 
     public Map<String, Object> getSimilarity() {
@@ -446,7 +446,7 @@ public class SchemaResponse extends SolrResponseBase {
     public void setResponse(NamedList<Object> response) {
       super.setResponse(response);
 
-      copyFields = SchemaResponse.getCopyFields(response);
+      copyFields = SchemaResponse.getCopyFields(response.asShallowMap());
     }
 
     public List<Map<String, Object>> getCopyFields() {
@@ -486,7 +486,7 @@ public class SchemaResponse extends SolrResponseBase {
     public void setResponse(NamedList<Object> response) {
       super.setResponse(response);
 
-      fieldTypes = SchemaResponse.getFieldTypeRepresentations(response);
+      fieldTypes = SchemaResponse.getFieldTypeRepresentations(response.asShallowMap());
     }
 
     public List<FieldTypeRepresentation> getFieldTypes() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/827c670d/solr/solrj/src/java/org/apache/solr/common/util/NamedList.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/NamedList.java b/solr/solrj/src/java/org/apache/solr/common/util/NamedList.java
index 3ca750d..dd5afe7 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/NamedList.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/NamedList.java
@@ -25,6 +25,7 @@ import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.solr.common.SolrException;
 
@@ -396,6 +397,74 @@ public class NamedList<T> implements Cloneable, Serializable, Iterable<Map.Entry
     return new NamedList<>( Collections.unmodifiableList(copy.nvPairs));
   }
 
+  public Map<String,T> asShallowMap() {
+    return new Map<String, T>() {
+      @Override
+      public int size() {
+        return NamedList.this.size();
+      }
+
+      @Override
+      public boolean isEmpty() {
+        return size() == 0;
+      }
+
+      public boolean containsKey(Object  key) {
+        return NamedList.this.get((String) key) != null ;
+      }
+
+      @Override
+      public boolean containsValue(Object value) {
+        return false;
+      }
+
+      @Override
+      public T get(Object key) {
+        return  NamedList.this.get((String) key);
+      }
+
+      @Override
+      public T put(String  key, T value) {
+        NamedList.this.add(key, value);
+        return  null;
+      }
+
+      @Override
+      public T remove(Object key) {
+        return  NamedList.this.remove((String) key);
+      }
+
+      @Override
+      public void putAll(Map m) {
+        NamedList.this.addAll(m);
+
+      }
+
+      @Override
+      public void clear() {
+        NamedList.this.clear();
+      }
+
+      @Override
+      public Set<String> keySet() {
+        //TODO implement more efficiently
+        return  NamedList.this.asMap(1).keySet();
+      }
+
+      @Override
+      public Collection values() {
+        //TODO implement more efficiently
+        return  NamedList.this.asMap(1).values();
+      }
+
+      @Override
+      public Set<Entry<String,T>> entrySet() {
+        //TODO implement more efficiently
+        return NamedList.this.asMap(1).entrySet();
+      }
+    };
+  }
+
   public Map asMap(int maxDepth) {
     LinkedHashMap result = new LinkedHashMap();
     for(int i=0;i<size();i++){


[46/50] [abbrv] lucene-solr:solr-5750: SOLR-5750: replace nocommit with TODO for testing custom properites

Posted by ds...@apache.org.
SOLR-5750: replace nocommit with TODO for testing custom properites


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

Branch: refs/heads/solr-5750
Commit: 6a9cc1adf69aab6c51a97b48aabc74cd2567e954
Parents: 66b56e6
Author: David Smiley <ds...@apache.org>
Authored: Wed Apr 27 11:13:54 2016 -0400
Committer: David Smiley <ds...@apache.org>
Committed: Wed Apr 27 11:13:54 2016 -0400

----------------------------------------------------------------------
 .../src/test/org/apache/solr/cloud/TestCloudBackupRestore.java    | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6a9cc1ad/solr/core/src/test/org/apache/solr/cloud/TestCloudBackupRestore.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestCloudBackupRestore.java b/solr/core/src/test/org/apache/solr/cloud/TestCloudBackupRestore.java
index 9354040..2f362d1 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestCloudBackupRestore.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestCloudBackupRestore.java
@@ -199,7 +199,8 @@ public class TestCloudBackupRestore extends SolrCloudTestCase {
         cluster.getSolrClient().getZkStateReader().readConfigName(restoreCollectionName));
 
     // assert added core properties:
-    // nocommit how?
+    // DWS: did via manual inspection.
+    // TODO Find the applicable core.properties on the file system but how?
   }
 
   private Map<String, Integer> getShardToDocCountMap(CloudSolrClient client, DocCollection docCollection) throws SolrServerException, IOException {


[15/50] [abbrv] lucene-solr:solr-5750: LUCENE-7237: LRUQueryCache now prefers returning an uncached Scorer than waiting on a lock.

Posted by ds...@apache.org.
LUCENE-7237: LRUQueryCache now prefers returning an uncached Scorer than waiting on a lock.


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

Branch: refs/heads/solr-5750
Commit: bf232d7635e1686cd6f5624525fa3e0b7820430f
Parents: 927a448
Author: Adrien Grand <jp...@gmail.com>
Authored: Fri Apr 22 14:09:44 2016 +0200
Committer: Adrien Grand <jp...@gmail.com>
Committed: Fri Apr 22 14:09:44 2016 +0200

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   3 +
 .../org/apache/lucene/search/LRUQueryCache.java | 234 ++++++++++++-------
 2 files changed, 154 insertions(+), 83 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bf232d76/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 18409c9..4b72294 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -76,6 +76,9 @@ Optimizations
 * LUCENE-7238: Explicitly disable the query cache in MemoryIndex#createSearcher.
   (Adrien Grand)
 
+* LUCENE-7237: LRUQueryCache now prefers returning an uncached Scorer than
+  waiting on a lock. (Adrien Grand)
+
 Bug Fixes
 
 * LUCENE-7127: Fix corner case bugs in GeoPointDistanceQuery. (Robert Muir)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bf232d76/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java b/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
index 15c0f2b..8598645 100644
--- a/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
+++ b/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
@@ -29,6 +29,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantLock;
 import java.util.function.Predicate;
 
 import org.apache.lucene.index.LeafReader.CoreClosedListener;
@@ -112,6 +113,7 @@ public class LRUQueryCache implements QueryCache, Accountable {
   // mostRecentlyUsedQueries. This is why write operations are performed under a lock
   private final Set<Query> mostRecentlyUsedQueries;
   private final Map<Object, LeafCache> cache;
+  private final ReentrantLock lock;
 
   // these variables are volatile so that we do not need to sync reads
   // but increments need to be performed under the lock
@@ -134,6 +136,7 @@ public class LRUQueryCache implements QueryCache, Accountable {
     uniqueQueries = new LinkedHashMap<>(16, 0.75f, true);
     mostRecentlyUsedQueries = uniqueQueries.keySet();
     cache = new IdentityHashMap<>();
+    lock = new ReentrantLock();
     ramBytesUsed = 0;
   }
 
@@ -182,6 +185,7 @@ public class LRUQueryCache implements QueryCache, Accountable {
    * @lucene.experimental
    */
   protected void onHit(Object readerCoreKey, Query query) {
+    assert lock.isHeldByCurrentThread();
     hitCount += 1;
   }
 
@@ -191,6 +195,7 @@ public class LRUQueryCache implements QueryCache, Accountable {
    * @lucene.experimental
    */
   protected void onMiss(Object readerCoreKey, Query query) {
+    assert lock.isHeldByCurrentThread();
     assert query != null;
     missCount += 1;
   }
@@ -203,6 +208,7 @@ public class LRUQueryCache implements QueryCache, Accountable {
    * @lucene.experimental
    */
   protected void onQueryCache(Query query, long ramBytesUsed) {
+    assert lock.isHeldByCurrentThread();
     this.ramBytesUsed += ramBytesUsed;
   }
 
@@ -212,6 +218,7 @@ public class LRUQueryCache implements QueryCache, Accountable {
    * @lucene.experimental
    */
   protected void onQueryEviction(Query query, long ramBytesUsed) {
+    assert lock.isHeldByCurrentThread();
     this.ramBytesUsed -= ramBytesUsed;
   }
 
@@ -223,6 +230,7 @@ public class LRUQueryCache implements QueryCache, Accountable {
    * @lucene.experimental
    */
   protected void onDocIdSetCache(Object readerCoreKey, long ramBytesUsed) {
+    assert lock.isHeldByCurrentThread();
     cacheSize += 1;
     cacheCount += 1;
     this.ramBytesUsed += ramBytesUsed;
@@ -235,6 +243,7 @@ public class LRUQueryCache implements QueryCache, Accountable {
    * @lucene.experimental
    */
   protected void onDocIdSetEviction(Object readerCoreKey, int numEntries, long sumRamBytesUsed) {
+    assert lock.isHeldByCurrentThread();
     this.ramBytesUsed -= sumRamBytesUsed;
     cacheSize -= numEntries;
   }
@@ -244,12 +253,14 @@ public class LRUQueryCache implements QueryCache, Accountable {
    * @lucene.experimental
    */
   protected void onClear() {
+    assert lock.isHeldByCurrentThread();
     ramBytesUsed = 0;
     cacheSize = 0;
   }
 
   /** Whether evictions are required. */
   boolean requiresEviction() {
+    assert lock.isHeldByCurrentThread();
     final int size = mostRecentlyUsedQueries.size();
     if (size == 0) {
       return false;
@@ -258,7 +269,8 @@ public class LRUQueryCache implements QueryCache, Accountable {
     }
   }
 
-  synchronized DocIdSet get(Query key, LeafReaderContext context) {
+  DocIdSet get(Query key, LeafReaderContext context) {
+    assert lock.isHeldByCurrentThread();
     assert key instanceof BoostQuery == false;
     assert key instanceof ConstantScoreQuery == false;
     final Object readerKey = context.reader().getCoreCacheKey();
@@ -282,40 +294,45 @@ public class LRUQueryCache implements QueryCache, Accountable {
     return cached;
   }
 
-  synchronized void putIfAbsent(Query query, LeafReaderContext context, DocIdSet set) {
-    // under a lock to make sure that mostRecentlyUsedQueries and cache remain sync'ed
-    // we don't want to have user-provided queries as keys in our cache since queries are mutable
+  void putIfAbsent(Query query, LeafReaderContext context, DocIdSet set) {
     assert query instanceof BoostQuery == false;
     assert query instanceof ConstantScoreQuery == false;
-    Query singleton = uniqueQueries.putIfAbsent(query, query);
-    if (singleton == null) {
-      onQueryCache(singleton, LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY + ramBytesUsed(query));
-    } else {
-      query = singleton;
-    }
-    final Object key = context.reader().getCoreCacheKey();
-    LeafCache leafCache = cache.get(key);
-    if (leafCache == null) {
-      leafCache = new LeafCache(key);
-      final LeafCache previous = cache.put(context.reader().getCoreCacheKey(), leafCache);
-      ramBytesUsed += HASHTABLE_RAM_BYTES_PER_ENTRY;
-      assert previous == null;
-      // we just created a new leaf cache, need to register a close listener
-      context.reader().addCoreClosedListener(new CoreClosedListener() {
-        @Override
-        public void onClose(Object ownerCoreCacheKey) {
-          clearCoreCacheKey(ownerCoreCacheKey);
-        }
-      });
+    // under a lock to make sure that mostRecentlyUsedQueries and cache remain sync'ed
+    lock.lock();
+    try {
+      Query singleton = uniqueQueries.putIfAbsent(query, query);
+      if (singleton == null) {
+        onQueryCache(singleton, LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY + ramBytesUsed(query));
+      } else {
+        query = singleton;
+      }
+      final Object key = context.reader().getCoreCacheKey();
+      LeafCache leafCache = cache.get(key);
+      if (leafCache == null) {
+        leafCache = new LeafCache(key);
+        final LeafCache previous = cache.put(context.reader().getCoreCacheKey(), leafCache);
+        ramBytesUsed += HASHTABLE_RAM_BYTES_PER_ENTRY;
+        assert previous == null;
+        // we just created a new leaf cache, need to register a close listener
+        context.reader().addCoreClosedListener(new CoreClosedListener() {
+          @Override
+          public void onClose(Object ownerCoreCacheKey) {
+            clearCoreCacheKey(ownerCoreCacheKey);
+          }
+        });
+      }
+      leafCache.putIfAbsent(query, set);
+      evictIfNecessary();
+    } finally {
+      lock.unlock();
     }
-    leafCache.putIfAbsent(query, set);
-    evictIfNecessary();
   }
 
-  synchronized void evictIfNecessary() {
+  void evictIfNecessary() {
+    assert lock.isHeldByCurrentThread();
     // under a lock to make sure that mostRecentlyUsedQueries and cache keep sync'ed
     if (requiresEviction()) {
-      
+
       Iterator<Query> iterator = mostRecentlyUsedQueries.iterator();
       do {
         final Query query = iterator.next();
@@ -337,31 +354,42 @@ public class LRUQueryCache implements QueryCache, Accountable {
   /**
    * Remove all cache entries for the given core cache key.
    */
-  public synchronized void clearCoreCacheKey(Object coreKey) {
-    final LeafCache leafCache = cache.remove(coreKey);
-    if (leafCache != null) {
-      ramBytesUsed -= HASHTABLE_RAM_BYTES_PER_ENTRY;
-      final int numEntries = leafCache.cache.size();
-      if (numEntries > 0) {
-        onDocIdSetEviction(coreKey, numEntries, leafCache.ramBytesUsed);
-      } else {
-        assert numEntries == 0;
-        assert leafCache.ramBytesUsed == 0;
+  public void clearCoreCacheKey(Object coreKey) {
+    lock.lock();
+    try {
+      final LeafCache leafCache = cache.remove(coreKey);
+      if (leafCache != null) {
+        ramBytesUsed -= HASHTABLE_RAM_BYTES_PER_ENTRY;
+        final int numEntries = leafCache.cache.size();
+        if (numEntries > 0) {
+          onDocIdSetEviction(coreKey, numEntries, leafCache.ramBytesUsed);
+        } else {
+          assert numEntries == 0;
+          assert leafCache.ramBytesUsed == 0;
+        }
       }
+    } finally {
+      lock.unlock();
     }
   }
 
   /**
    * Remove all cache entries for the given query.
    */
-  public synchronized void clearQuery(Query query) {
-    final Query singleton = uniqueQueries.remove(query);
-    if (singleton != null) {
-      onEviction(singleton);
+  public void clearQuery(Query query) {
+    lock.lock();
+    try {
+      final Query singleton = uniqueQueries.remove(query);
+      if (singleton != null) {
+        onEviction(singleton);
+      }
+    } finally {
+      lock.unlock();
     }
   }
 
   private void onEviction(Query singleton) {
+    assert lock.isHeldByCurrentThread();
     onQueryEviction(singleton, LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY + ramBytesUsed(singleton));
     for (LeafCache leafCache : cache.values()) {
       leafCache.remove(singleton);
@@ -371,55 +399,70 @@ public class LRUQueryCache implements QueryCache, Accountable {
   /**
    * Clear the content of this cache.
    */
-  public synchronized void clear() {
-    cache.clear();
-    mostRecentlyUsedQueries.clear();
-    onClear();
+  public void clear() {
+    lock.lock();
+    try {
+      cache.clear();
+      mostRecentlyUsedQueries.clear();
+      onClear();
+    } finally {
+      lock.unlock();
+    }
   }
 
   // pkg-private for testing
-  synchronized void assertConsistent() {
-    if (requiresEviction()) {
-      throw new AssertionError("requires evictions: size=" + mostRecentlyUsedQueries.size()
-          + ", maxSize=" + maxSize + ", ramBytesUsed=" + ramBytesUsed() + ", maxRamBytesUsed=" + maxRamBytesUsed);
-    }
-    for (LeafCache leafCache : cache.values()) {
-      Set<Query> keys = Collections.newSetFromMap(new IdentityHashMap<>());
-      keys.addAll(leafCache.cache.keySet());
-      keys.removeAll(mostRecentlyUsedQueries);
-      if (!keys.isEmpty()) {
-        throw new AssertionError("One leaf cache contains more keys than the top-level cache: " + keys);
+  void assertConsistent() {
+    lock.lock();
+    try {
+      if (requiresEviction()) {
+        throw new AssertionError("requires evictions: size=" + mostRecentlyUsedQueries.size()
+            + ", maxSize=" + maxSize + ", ramBytesUsed=" + ramBytesUsed() + ", maxRamBytesUsed=" + maxRamBytesUsed);
       }
-    }
-    long recomputedRamBytesUsed =
-          HASHTABLE_RAM_BYTES_PER_ENTRY * cache.size()
-        + LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY * uniqueQueries.size();
-    for (Query query : mostRecentlyUsedQueries) {
-      recomputedRamBytesUsed += ramBytesUsed(query);
-    }
-    for (LeafCache leafCache : cache.values()) {
-      recomputedRamBytesUsed += HASHTABLE_RAM_BYTES_PER_ENTRY * leafCache.cache.size();
-      for (DocIdSet set : leafCache.cache.values()) {
-        recomputedRamBytesUsed += set.ramBytesUsed();
+      for (LeafCache leafCache : cache.values()) {
+        Set<Query> keys = Collections.newSetFromMap(new IdentityHashMap<>());
+        keys.addAll(leafCache.cache.keySet());
+        keys.removeAll(mostRecentlyUsedQueries);
+        if (!keys.isEmpty()) {
+          throw new AssertionError("One leaf cache contains more keys than the top-level cache: " + keys);
+        }
+      }
+      long recomputedRamBytesUsed =
+            HASHTABLE_RAM_BYTES_PER_ENTRY * cache.size()
+          + LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY * uniqueQueries.size();
+      for (Query query : mostRecentlyUsedQueries) {
+        recomputedRamBytesUsed += ramBytesUsed(query);
+      }
+      for (LeafCache leafCache : cache.values()) {
+        recomputedRamBytesUsed += HASHTABLE_RAM_BYTES_PER_ENTRY * leafCache.cache.size();
+        for (DocIdSet set : leafCache.cache.values()) {
+          recomputedRamBytesUsed += set.ramBytesUsed();
+        }
+      }
+      if (recomputedRamBytesUsed != ramBytesUsed) {
+        throw new AssertionError("ramBytesUsed mismatch : " + ramBytesUsed + " != " + recomputedRamBytesUsed);
       }
-    }
-    if (recomputedRamBytesUsed != ramBytesUsed) {
-      throw new AssertionError("ramBytesUsed mismatch : " + ramBytesUsed + " != " + recomputedRamBytesUsed);
-    }
 
-    long recomputedCacheSize = 0;
-    for (LeafCache leafCache : cache.values()) {
-      recomputedCacheSize += leafCache.cache.size();
-    }
-    if (recomputedCacheSize != getCacheSize()) {
-      throw new AssertionError("cacheSize mismatch : " + getCacheSize() + " != " + recomputedCacheSize);
+      long recomputedCacheSize = 0;
+      for (LeafCache leafCache : cache.values()) {
+        recomputedCacheSize += leafCache.cache.size();
+      }
+      if (recomputedCacheSize != getCacheSize()) {
+        throw new AssertionError("cacheSize mismatch : " + getCacheSize() + " != " + recomputedCacheSize);
+      }
+    } finally {
+      lock.unlock();
     }
   }
 
   // pkg-private for testing
   // return the list of cached queries in LRU order
-  synchronized List<Query> cachedQueries() {
-    return new ArrayList<>(mostRecentlyUsedQueries);
+  List<Query> cachedQueries() {
+    lock.lock();
+    try {
+      return new ArrayList<>(mostRecentlyUsedQueries);
+    } finally {
+      lock.unlock();
+    }
   }
 
   @Override
@@ -438,8 +481,11 @@ public class LRUQueryCache implements QueryCache, Accountable {
 
   @Override
   public Collection<Accountable> getChildResources() {
-    synchronized (this) {
+    lock.lock();
+    try {
       return Accountables.namedAccountables("segment", cache);
+    } finally {
+      lock.unlock();
     }
   }
 
@@ -659,7 +705,18 @@ public class LRUQueryCache implements QueryCache, Accountable {
         return in.scorer(context);
       }
 
-      DocIdSet docIdSet = get(in.getQuery(), context);
+      // If the lock is already busy, prefer using the uncached version than waiting
+      if (lock.tryLock() == false) {
+        return in.scorer(context);
+      }
+
+      DocIdSet docIdSet;
+      try {
+        docIdSet = get(in.getQuery(), context);
+      } finally {
+        lock.unlock();
+      }
+
       if (docIdSet == null) {
         if (policy.shouldCache(in.getQuery())) {
           docIdSet = cache(context);
@@ -692,7 +749,18 @@ public class LRUQueryCache implements QueryCache, Accountable {
         return in.bulkScorer(context);
       }
 
-      DocIdSet docIdSet = get(in.getQuery(), context);
+      // If the lock is already busy, prefer using the uncached version than waiting
+      if (lock.tryLock() == false) {
+        return in.bulkScorer(context);
+      }
+
+      DocIdSet docIdSet;
+      try {
+        docIdSet = get(in.getQuery(), context);
+      } finally {
+        lock.unlock();
+      }
+
       if (docIdSet == null) {
         if (policy.shouldCache(in.getQuery())) {
           docIdSet = cache(context);


[09/50] [abbrv] lucene-solr:solr-5750: LUCENE-7186: add a numerically stable morton encoding method to GeoEncodingUtils

Posted by ds...@apache.org.
LUCENE-7186: add a numerically stable morton encoding method to GeoEncodingUtils


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

Branch: refs/heads/solr-5750
Commit: 545a14000bf4c02f25d6a3a4a52d51326199965f
Parents: dac6fe0
Author: nknize <nk...@apache.org>
Authored: Mon Apr 11 13:53:27 2016 -0500
Committer: nknize <nk...@apache.org>
Committed: Thu Apr 21 10:14:36 2016 -0500

----------------------------------------------------------------------
 .../java/org/apache/lucene/util/BitUtil.java    |   4 +-
 .../geopoint/document/GeoPointField.java        | 110 +++++++++++++-
 .../geopoint/document/GeoPointTokenStream.java  |   4 +-
 .../geopoint/search/GeoPointInPolygonQuery.java |   1 -
 .../search/GeoPointNumericTermsEnum.java        |  17 ++-
 .../search/GeoPointPrefixTermsEnum.java         |  26 ++--
 .../GeoPointTermQueryConstantScoreWrapper.java  |   7 +-
 .../lucene/spatial/util/GeoEncodingUtils.java   | 146 -------------------
 .../lucene/spatial/util/MortonEncoder.java      | 103 +++++++++++++
 .../geopoint/search/TestGeoPointQuery.java      |   5 +-
 .../search/TestLegacyGeoPointQuery.java         |   5 +-
 .../spatial/util/TestGeoEncodingUtils.java      | 102 -------------
 .../lucene/spatial/util/TestGeoPointField.java  |  51 +++++++
 .../lucene/spatial/util/TestMortonEncoder.java  | 108 ++++++++++++++
 14 files changed, 397 insertions(+), 292 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/545a1400/lucene/core/src/java/org/apache/lucene/util/BitUtil.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/BitUtil.java b/lucene/core/src/java/org/apache/lucene/util/BitUtil.java
index 90f4f17..404ce7f 100644
--- a/lucene/core/src/java/org/apache/lucene/util/BitUtil.java
+++ b/lucene/core/src/java/org/apache/lucene/util/BitUtil.java
@@ -116,7 +116,9 @@ public final class BitUtil {
    *
    * Adapted from: http://graphics.stanford.edu/~seander/bithacks.html#InterleaveBMN
    */
-  public static long interleave(long v1, long v2) {
+  public static long interleave(int even, int odd) {
+    long v1 = 0x00000000FFFFFFFFL & even;
+    long v2 = 0x00000000FFFFFFFFL & odd;
     v1 = (v1 | (v1 << SHIFT[4])) & MAGIC[4];
     v1 = (v1 | (v1 << SHIFT[3])) & MAGIC[3];
     v1 = (v1 | (v1 << SHIFT[2])) & MAGIC[2];

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/545a1400/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/document/GeoPointField.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/document/GeoPointField.java b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/document/GeoPointField.java
index 8b1483a..c20ad85 100644
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/document/GeoPointField.java
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/document/GeoPointField.java
@@ -22,8 +22,14 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.index.IndexOptions;
-import org.apache.lucene.spatial.util.GeoEncodingUtils;
 import org.apache.lucene.geo.GeoUtils;
+import org.apache.lucene.util.BitUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+
+import static org.apache.lucene.spatial.util.MortonEncoder.encode;
+import static org.apache.lucene.geo.GeoUtils.MIN_LAT_INCL;
+import static org.apache.lucene.geo.GeoUtils.MIN_LON_INCL;
 
 /**
  * <p>
@@ -49,6 +55,19 @@ public final class GeoPointField extends Field {
   /** encoding step value for GeoPoint prefix terms */
   public static final int PRECISION_STEP = 9;
 
+  /** number of bits used for quantizing latitude and longitude values */
+  public static final short BITS = 31;
+  /** scaling factors to convert lat/lon into unsigned space */
+  private static final double LAT_SCALE = (0x1L<<BITS)/180.0D;
+  private static final double LON_SCALE = (0x1L<<BITS)/360.0D;
+
+  /**
+   * The maximum term length (used for <code>byte[]</code> buffer size)
+   * for encoding <code>geoEncoded</code> values.
+   * @see #geoCodedToPrefixCodedBytes(long, int, BytesRefBuilder)
+   */
+  private static final int BUF_SIZE_LONG = 28/8 + 1;
+
   /**
    * <b>Expert:</b> Optional flag to select term encoding for GeoPointField types
    */
@@ -185,7 +204,7 @@ public final class GeoPointField extends Field {
     }
 
     // set field data
-    fieldsData = GeoEncodingUtils.mortonHash(latitude, longitude);
+    fieldsData = encodeLatLon(latitude, longitude);
   }
 
   private static FieldType getFieldType(Store stored) {
@@ -232,12 +251,12 @@ public final class GeoPointField extends Field {
 
   /** access latitude value */
   public double getLat() {
-    return GeoEncodingUtils.mortonUnhashLat((long) fieldsData);
+    return decodeLatitude((long) fieldsData);
   }
 
   /** access longitude value */
   public double getLon() {
-    return GeoEncodingUtils.mortonUnhashLon((long) fieldsData);
+    return decodeLongitude((long) fieldsData);
   }
 
   @Override
@@ -246,9 +265,88 @@ public final class GeoPointField extends Field {
       return null;
     }
     StringBuilder sb = new StringBuilder();
-    sb.append(GeoEncodingUtils.mortonUnhashLat((long) fieldsData));
+    sb.append(decodeLatitude((long) fieldsData));
     sb.append(',');
-    sb.append(GeoEncodingUtils.mortonUnhashLon((long) fieldsData));
+    sb.append(decodeLongitude((long) fieldsData));
     return sb.toString();
   }
+
+  /*************************
+   * 31 bit encoding utils *
+   *************************/
+  public static long encodeLatLon(final double lat, final double lon) {
+    long result = encode(lat, lon);
+    if (result == 0xFFFFFFFFFFFFFFFFL) {
+      return result & 0xC000000000000000L;
+    }
+    return result >>> 2;
+  }
+
+  /** decode longitude value from morton encoded geo point */
+  public static final double decodeLongitude(final long hash) {
+    return unscaleLon(BitUtil.deinterleave(hash));
+  }
+
+  /** decode latitude value from morton encoded geo point */
+  public static final double decodeLatitude(final long hash) {
+    return unscaleLat(BitUtil.deinterleave(hash >>> 1));
+  }
+
+  private static final double unscaleLon(final long val) {
+    return (val / LON_SCALE) + MIN_LON_INCL;
+  }
+
+  private static final double unscaleLat(final long val) {
+    return (val / LAT_SCALE) + MIN_LAT_INCL;
+  }
+
+  /** Convert a geocoded morton long into a prefix coded geo term */
+  public static void geoCodedToPrefixCoded(long hash, int shift, BytesRefBuilder bytes) {
+    geoCodedToPrefixCodedBytes(hash, shift, bytes);
+  }
+
+  /** Convert a prefix coded geo term back into the geocoded morton long */
+  public static long prefixCodedToGeoCoded(final BytesRef val) {
+    final long result = 0L
+        | (val.bytes[val.offset+0] & 255L) << 24
+        | (val.bytes[val.offset+1] & 255L) << 16
+        | (val.bytes[val.offset+2] & 255L) << 8
+        | val.bytes[val.offset+3] & 255L;
+
+    return result << 32;
+  }
+
+  /**
+   * GeoTerms are coded using 4 prefix bytes + 1 byte to record number of prefix bits
+   *
+   * example prefix at shift 54 (yields 10 significant prefix bits):
+   *  pppppppp pp000000 00000000 00000000 00001010
+   *  (byte 1) (byte 2) (byte 3) (byte 4) (sigbits)
+   */
+  private static void geoCodedToPrefixCodedBytes(final long hash, final int shift, final BytesRefBuilder bytes) {
+    // ensure shift is 32..63
+    if (shift < 32 || shift > 63) {
+      throw new IllegalArgumentException("Illegal shift value, must be 32..63; got shift=" + shift);
+    }
+    int nChars = BUF_SIZE_LONG + 1; // one extra for the byte that contains the number of significant bits
+    bytes.setLength(nChars);
+    bytes.grow(nChars--);
+    final int sigBits = 64 - shift;
+    bytes.setByteAt(BUF_SIZE_LONG, (byte)(sigBits));
+    long sortableBits = hash;
+    sortableBits >>>= shift;
+    sortableBits <<= 32 - sigBits;
+    do {
+      bytes.setByteAt(--nChars, (byte)(sortableBits));
+      sortableBits >>>= 8;
+    } while (nChars > 0);
+  }
+
+  /** Get the prefix coded geo term shift value */
+  public static int getPrefixCodedShift(final BytesRef val) {
+    final int shift = val.bytes[val.offset + BUF_SIZE_LONG];
+    if (shift > 63 || shift < 0)
+      throw new NumberFormatException("Invalid shift value (" + shift + ") in prefixCoded bytes (is encoded value really a geo point?)");
+    return shift;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/545a1400/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/document/GeoPointTokenStream.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/document/GeoPointTokenStream.java b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/document/GeoPointTokenStream.java
index c374c4e..cc5d8f2 100644
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/document/GeoPointTokenStream.java
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/document/GeoPointTokenStream.java
@@ -29,8 +29,8 @@ import org.apache.lucene.util.AttributeImpl;
 import org.apache.lucene.util.AttributeReflector;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.lucene.spatial.util.GeoEncodingUtils;
 
+import static org.apache.lucene.spatial.geopoint.document.GeoPointField.geoCodedToPrefixCoded;
 import static org.apache.lucene.spatial.geopoint.document.GeoPointField.PRECISION_STEP;
 
 /**
@@ -163,7 +163,7 @@ final class GeoPointTokenStream extends TokenStream {
 
     @Override
     public BytesRef getBytesRef() {
-      GeoEncodingUtils.geoCodedToPrefixCoded(value, shift, bytes);
+      geoCodedToPrefixCoded(value, shift, bytes);
       return bytes.get();
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/545a1400/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointInPolygonQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointInPolygonQuery.java b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointInPolygonQuery.java
index 26af78e..7a0a9e9 100644
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointInPolygonQuery.java
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointInPolygonQuery.java
@@ -22,7 +22,6 @@ import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.spatial.geopoint.document.GeoPointField;
 import org.apache.lucene.spatial.geopoint.document.GeoPointField.TermEncoding;
-import org.apache.lucene.spatial.util.GeoEncodingUtils;
 import org.apache.lucene.geo.Rectangle;
 import org.apache.lucene.geo.Polygon;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/545a1400/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointNumericTermsEnum.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointNumericTermsEnum.java b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointNumericTermsEnum.java
index 071ab1e..06dfec7 100644
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointNumericTermsEnum.java
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointNumericTermsEnum.java
@@ -23,11 +23,12 @@ import java.util.List;
 
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.spatial.geopoint.document.GeoPointField;
-import org.apache.lucene.spatial.util.GeoEncodingUtils;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.LegacyNumericUtils;
 
+import static org.apache.lucene.spatial.geopoint.document.GeoPointField.BITS;
+
 /**
  * Decomposes a given {@link GeoPointMultiTermQuery} into a set of terms that represent the query criteria using
  * {@link org.apache.lucene.spatial.geopoint.document.GeoPointField.TermEncoding#NUMERIC} method defined by
@@ -46,8 +47,8 @@ final class GeoPointNumericTermsEnum extends GeoPointTermsEnum {
 
   GeoPointNumericTermsEnum(final TermsEnum tenum, final GeoPointMultiTermQuery query) {
     super(tenum, query);
-    DETAIL_LEVEL = (short)(((GeoEncodingUtils.BITS<<1)-this.maxShift)/2);
-    computeRange(0L, (short) (((GeoEncodingUtils.BITS) << 1) - 1));
+    DETAIL_LEVEL = (short)(((BITS<<1)-this.maxShift)/2);
+    computeRange(0L, (short) (((BITS) << 1) - 1));
     assert rangeBounds.isEmpty() == false;
     Collections.sort(rangeBounds);
   }
@@ -79,12 +80,12 @@ final class GeoPointNumericTermsEnum extends GeoPointTermsEnum {
    * @param res spatial res represented as a bit shift (MSB is lower res)
    */
   private void relateAndRecurse(final long start, final long end, final short res) {
-    final double minLon = GeoEncodingUtils.mortonUnhashLon(start);
-    final double minLat = GeoEncodingUtils.mortonUnhashLat(start);
-    final double maxLon = GeoEncodingUtils.mortonUnhashLon(end);
-    final double maxLat = GeoEncodingUtils.mortonUnhashLat(end);
+    final double minLon = GeoPointField.decodeLongitude(start);
+    final double minLat = GeoPointField.decodeLatitude(start);
+    final double maxLon = GeoPointField.decodeLongitude(end);
+    final double maxLat = GeoPointField.decodeLatitude(end);
 
-    final short level = (short)((GeoEncodingUtils.BITS<<1)-res>>>1);
+    final short level = (short)((BITS<<1)-res>>>1);
 
     // if cell is within and a factor of the precision step, or it crosses the edge of the shape add the range
     final boolean within = res % GeoPointField.PRECISION_STEP == 0 && relationImpl.cellWithin(minLat, maxLat, minLon, maxLon);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/545a1400/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointPrefixTermsEnum.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointPrefixTermsEnum.java b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointPrefixTermsEnum.java
index f511392..6429b92 100644
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointPrefixTermsEnum.java
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointPrefixTermsEnum.java
@@ -21,14 +21,10 @@ import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.spatial.geopoint.document.GeoPointField;
-import org.apache.lucene.spatial.util.GeoEncodingUtils;
 
-import static org.apache.lucene.spatial.util.GeoEncodingUtils.mortonHash;
-import static org.apache.lucene.spatial.util.GeoEncodingUtils.mortonUnhashLat;
-import static org.apache.lucene.spatial.util.GeoEncodingUtils.mortonUnhashLon;
-import static org.apache.lucene.spatial.util.GeoEncodingUtils.geoCodedToPrefixCoded;
-import static org.apache.lucene.spatial.util.GeoEncodingUtils.prefixCodedToGeoCoded;
-import static org.apache.lucene.spatial.util.GeoEncodingUtils.getPrefixCodedShift;
+import static org.apache.lucene.spatial.geopoint.document.GeoPointField.geoCodedToPrefixCoded;
+import static org.apache.lucene.spatial.geopoint.document.GeoPointField.prefixCodedToGeoCoded;
+import static org.apache.lucene.spatial.geopoint.document.GeoPointField.getPrefixCodedShift;
 
 /**
  * Decomposes a given {@link GeoPointMultiTermQuery} into a set of terms that represent the query criteria using
@@ -58,7 +54,7 @@ final class GeoPointPrefixTermsEnum extends GeoPointTermsEnum {
 
   public GeoPointPrefixTermsEnum(final TermsEnum tenum, final GeoPointMultiTermQuery query) {
     super(tenum, query);
-    this.start = mortonHash(query.minLat, query.minLon);
+    this.start = GeoPointField.encodeLatLon(query.minLat, query.minLon);
     this.currentRange = new Range(0, shift, true);
     // start shift at maxShift value (from computeMaxShift)
     this.shift = maxShift;
@@ -90,14 +86,14 @@ final class GeoPointPrefixTermsEnum extends GeoPointTermsEnum {
   }
 
   private void nextRelation() {
-    double minLon = mortonUnhashLon(currStart);
-    double minLat = mortonUnhashLat(currStart);
+    double minLon = GeoPointField.decodeLongitude(currStart);
+    double minLat = GeoPointField.decodeLatitude(currStart);
     double maxLon;
     double maxLat;
     boolean isWithin;
     do {
-      maxLon = mortonUnhashLon(currEnd);
-      maxLat = mortonUnhashLat(currEnd);
+      maxLon = GeoPointField.decodeLongitude(currEnd);
+      maxLat = GeoPointField.decodeLatitude(currEnd);
 
       isWithin = false;
       // within or a boundary
@@ -123,8 +119,8 @@ final class GeoPointPrefixTermsEnum extends GeoPointTermsEnum {
         currEnd = currStart | (1L<<--shift) - 1;
       } else {
         advanceVariables();
-        minLon = mortonUnhashLon(currStart);
-        minLat = mortonUnhashLat(currStart);
+        minLon = GeoPointField.decodeLongitude(currStart);
+        minLat = GeoPointField.decodeLatitude(currStart);
       }
     } while(shift < 63);
   }
@@ -194,7 +190,7 @@ final class GeoPointPrefixTermsEnum extends GeoPointTermsEnum {
 
       final int comparison = term.compareTo(currentCell);
       if (comparison > 0) {
-        seek(GeoEncodingUtils.prefixCodedToGeoCoded(term), (short)(64-GeoEncodingUtils.getPrefixCodedShift(term)));
+        seek(prefixCodedToGeoCoded(term), (short)(64 - getPrefixCodedShift(term)));
         continue;
       }
       return currentCell;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/545a1400/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointTermQueryConstantScoreWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointTermQueryConstantScoreWrapper.java b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointTermQueryConstantScoreWrapper.java
index 7e97f16..96e0bd9 100644
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointTermQueryConstantScoreWrapper.java
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointTermQueryConstantScoreWrapper.java
@@ -32,15 +32,12 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.TwoPhaseIterator;
 import org.apache.lucene.search.Weight;
+import org.apache.lucene.spatial.geopoint.document.GeoPointField;
 import org.apache.lucene.util.BitSet;
 import org.apache.lucene.util.DocIdSetBuilder;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.SparseFixedBitSet;
 
-import static org.apache.lucene.spatial.util.GeoEncodingUtils.mortonUnhashLat;
-import static org.apache.lucene.spatial.util.GeoEncodingUtils.mortonUnhashLon;
-
-
 /**
  * Custom ConstantScoreWrapper for {@code GeoPointMultiTermQuery} that cuts over to DocValues
  * for post filtering boundary ranges. Multi-valued GeoPoint documents are supported.
@@ -140,7 +137,7 @@ final class GeoPointTermQueryConstantScoreWrapper <Q extends GeoPointMultiTermQu
               int count = sdv.count();
               for (int i = 0; i < count; i++) {
                 long hash = sdv.valueAt(i);
-                if (termsEnum.postFilter(mortonUnhashLat(hash), mortonUnhashLon(hash))) {
+                if (termsEnum.postFilter(GeoPointField.decodeLatitude(hash), GeoPointField.decodeLongitude(hash))) {
                   return true;
                 }
               }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/545a1400/lucene/spatial/src/java/org/apache/lucene/spatial/util/GeoEncodingUtils.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/util/GeoEncodingUtils.java b/lucene/spatial/src/java/org/apache/lucene/spatial/util/GeoEncodingUtils.java
deleted file mode 100644
index 95aea2a..0000000
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/util/GeoEncodingUtils.java
+++ /dev/null
@@ -1,146 +0,0 @@
-package org.apache.lucene.spatial.util;
-
-/*
- * 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.
- */
-
-import org.apache.lucene.util.BitUtil;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefBuilder;
-
-import static org.apache.lucene.geo.GeoUtils.MIN_LON_INCL;
-import static org.apache.lucene.geo.GeoUtils.MIN_LAT_INCL;
-
-/**
- * Basic reusable geopoint encoding methods
- *
- * @lucene.experimental
- */
-public final class GeoEncodingUtils {
-  /** number of bits used for quantizing latitude and longitude values */
-  public static final short BITS = 31;
-
-  private static final double LAT_SCALE = (0x1L<<BITS)/180.0D;
-  private static final double LON_SCALE = (0x1L<<BITS)/360.0D;
-
-  /**
-   * The maximum term length (used for <code>byte[]</code> buffer size)
-   * for encoding <code>geoEncoded</code> values.
-   * @see #geoCodedToPrefixCodedBytes(long, int, BytesRefBuilder)
-   */
-  private static final int BUF_SIZE_LONG = 28/8 + 1;
-
-  // No instance:
-  private GeoEncodingUtils() {
-  }
-
-  /**
-   * encode latitude, longitude geopoint values using morton encoding method
-   * https://en.wikipedia.org/wiki/Z-order_curve
-   */
-  public static final Long mortonHash(final double lat, final double lon) {
-    return BitUtil.interleave(scaleLon(lon), scaleLat(lat));
-  }
-
-  /** decode longitude value from morton encoded geo point */
-  public static final double mortonUnhashLon(final long hash) {
-    return unscaleLon(BitUtil.deinterleave(hash));
-  }
-
-  /** decode latitude value from morton encoded geo point */
-  public static final double mortonUnhashLat(final long hash) {
-    return unscaleLat(BitUtil.deinterleave(hash >>> 1));
-  }
-
-  private static final long scaleLon(final double val) {
-    return (long) ((val-MIN_LON_INCL) * LON_SCALE);
-  }
-
-  private static final long scaleLat(final double val) {
-    return (long) ((val-MIN_LAT_INCL) * LAT_SCALE);
-  }
-
-  private static final double unscaleLon(final long val) {
-    return (val / LON_SCALE) + MIN_LON_INCL;
-  }
-
-  private static final double unscaleLat(final long val) {
-    return (val / LAT_SCALE) + MIN_LAT_INCL;
-  }
-
-  /** Convert a geocoded morton long into a prefix coded geo term */
-  public static void geoCodedToPrefixCoded(long hash, int shift, BytesRefBuilder bytes) {
-    geoCodedToPrefixCodedBytes(hash, shift, bytes);
-  }
-
-  /** Convert a prefix coded geo term back into the geocoded morton long */
-  public static long prefixCodedToGeoCoded(final BytesRef val) {
-    final long result = 0L
-        | (val.bytes[val.offset+0] & 255L) << 24
-        | (val.bytes[val.offset+1] & 255L) << 16
-        | (val.bytes[val.offset+2] & 255L) << 8
-        | val.bytes[val.offset+3] & 255L;
-
-    return result << 32;
-  }
-
-  /**
-   * GeoTerms are coded using 4 prefix bytes + 1 byte to record number of prefix bits
-   *
-   * example prefix at shift 54 (yields 10 significant prefix bits):
-   *  pppppppp pp000000 00000000 00000000 00001010
-   *  (byte 1) (byte 2) (byte 3) (byte 4) (sigbits)
-   */
-  private static void geoCodedToPrefixCodedBytes(final long hash, final int shift, final BytesRefBuilder bytes) {
-    // ensure shift is 32..63
-    if (shift < 32 || shift > 63) {
-      throw new IllegalArgumentException("Illegal shift value, must be 32..63; got shift=" + shift);
-    }
-    int nChars = BUF_SIZE_LONG + 1; // one extra for the byte that contains the number of significant bits
-    bytes.setLength(nChars);
-    bytes.grow(nChars--);
-    final int sigBits = 64 - shift;
-    bytes.setByteAt(BUF_SIZE_LONG, (byte)(sigBits));
-    long sortableBits = hash;
-    sortableBits >>>= shift;
-    sortableBits <<= 32 - sigBits;
-    do {
-      bytes.setByteAt(--nChars, (byte)(sortableBits));
-      sortableBits >>>= 8;
-    } while (nChars > 0);
-  }
-
-  /** Get the prefix coded geo term shift value */
-  public static int getPrefixCodedShift(final BytesRef val) {
-    final int shift = val.bytes[val.offset + BUF_SIZE_LONG];
-    if (shift > 63 || shift < 0)
-      throw new NumberFormatException("Invalid shift value (" + shift + ") in prefixCoded bytes (is encoded value really a geo point?)");
-    return shift;
-  }
-
-  /** Converts a long value into a bit string (useful for debugging) */
-  public static String geoTermToString(long term) {
-    StringBuilder s = new StringBuilder(64);
-    final int numberOfLeadingZeros = Long.numberOfLeadingZeros(term);
-    for (int i = 0; i < numberOfLeadingZeros; i++) {
-      s.append('0');
-    }
-    if (term != 0) {
-      s.append(Long.toBinaryString(term));
-    }
-    return s.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/545a1400/lucene/spatial/src/java/org/apache/lucene/spatial/util/MortonEncoder.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/util/MortonEncoder.java b/lucene/spatial/src/java/org/apache/lucene/spatial/util/MortonEncoder.java
new file mode 100644
index 0000000..2340259
--- /dev/null
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/util/MortonEncoder.java
@@ -0,0 +1,103 @@
+/*
+ * 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.spatial.util;
+
+import org.apache.lucene.geo.GeoEncodingUtils;
+import org.apache.lucene.util.BitUtil;
+
+import static org.apache.lucene.geo.GeoUtils.checkLatitude;
+import static org.apache.lucene.geo.GeoUtils.checkLongitude;
+import static org.apache.lucene.geo.GeoEncodingUtils.encodeLatitude;
+import static org.apache.lucene.geo.GeoEncodingUtils.encodeLatitudeCeil;
+import static org.apache.lucene.geo.GeoEncodingUtils.encodeLongitude;
+import static org.apache.lucene.geo.GeoEncodingUtils.encodeLongitudeCeil;
+
+/**
+ * Quantizes lat/lon points and bit interleaves them into a binary morton code
+ * in the range of 0x00000000... : 0xFFFFFFFF...
+ * https://en.wikipedia.org/wiki/Z-order_curve
+ *
+ * This is useful for bitwise operations in raster space
+ *
+ * @lucene.experimental
+ */
+public class MortonEncoder {
+
+  private MortonEncoder() {} // no instance
+
+  /**
+   * Main encoding method to quantize lat/lon points and bit interleave them into a binary morton code
+   * in the range of 0x00000000... : 0xFFFFFFFF...
+   *
+   * @param latitude latitude value: must be within standard +/-90 coordinate bounds.
+   * @param longitude longitude value: must be within standard +/-180 coordinate bounds.
+   * @return bit interleaved encoded values as a 64-bit {@code long}
+   * @throws IllegalArgumentException if latitude or longitude is out of bounds
+   */
+  public static final long encode(double latitude, double longitude) {
+    checkLatitude(latitude);
+    checkLongitude(longitude);
+    // encode lat/lon flipping the sign bit so negative ints sort before positive ints
+    final int latEnc = encodeLatitude(latitude) ^ 0x80000000;
+    final int lonEnc = encodeLongitude(longitude) ^ 0x80000000;
+    return BitUtil.interleave(lonEnc, latEnc);
+  }
+
+  /**
+   * Quantizes lat/lon points and bit interleaves them into a sortable morton code
+   * ranging from 0x00 : 0xFF...
+   * https://en.wikipedia.org/wiki/Z-order_curve
+   * This is useful for bitwise operations in raster space
+   * @param latitude latitude value: must be within standard +/-90 coordinate bounds.
+   * @param longitude longitude value: must be within standard +/-180 coordinate bounds.
+   * @return bit interleaved encoded values as a 64-bit {@code long}
+   * @throws IllegalArgumentException if latitude or longitude is out of bounds
+   */
+  public static final long encodeCeil(double latitude, double longitude) {
+    checkLatitude(latitude);
+    checkLongitude(longitude);
+    // encode lat/lon flipping the sign bit so negative ints sort before positive ints
+    final int latEnc = encodeLatitudeCeil(latitude) ^ 0x80000000;
+    final int lonEnc = encodeLongitudeCeil(longitude) ^ 0x80000000;
+    return BitUtil.interleave(lonEnc, latEnc);
+  }
+
+  /** decode latitude value from morton encoded geo point */
+  public static final double decodeLatitude(final long hash) {
+    // decode lat/lon flipping the sign bit so negative ints sort before positive ints
+    return GeoEncodingUtils.decodeLatitude((int) BitUtil.deinterleave(hash >>> 1) ^ 0x80000000);
+  }
+
+  /** decode longitude value from morton encoded geo point */
+  public static final double decodeLongitude(final long hash) {
+    // decode lat/lon flipping the sign bit so negative ints sort before positive ints
+    return GeoEncodingUtils.decodeLongitude((int) BitUtil.deinterleave(hash) ^ 0x80000000);
+  }
+
+  /** Converts a long value into a full 64 bit string (useful for debugging) */
+  public static String geoTermToString(long term) {
+    StringBuilder s = new StringBuilder(64);
+    final int numberOfLeadingZeros = Long.numberOfLeadingZeros(term);
+    for (int i = 0; i < numberOfLeadingZeros; i++) {
+      s.append('0');
+    }
+    if (term != 0) {
+      s.append(Long.toBinaryString(term));
+    }
+    return s.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/545a1400/lucene/spatial/src/test/org/apache/lucene/spatial/geopoint/search/TestGeoPointQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/test/org/apache/lucene/spatial/geopoint/search/TestGeoPointQuery.java b/lucene/spatial/src/test/org/apache/lucene/spatial/geopoint/search/TestGeoPointQuery.java
index ab6b646..cfd6630 100644
--- a/lucene/spatial/src/test/org/apache/lucene/spatial/geopoint/search/TestGeoPointQuery.java
+++ b/lucene/spatial/src/test/org/apache/lucene/spatial/geopoint/search/TestGeoPointQuery.java
@@ -18,7 +18,6 @@ package org.apache.lucene.spatial.geopoint.search;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.search.Query;
-import org.apache.lucene.spatial.util.GeoEncodingUtils;
 import org.apache.lucene.geo.BaseGeoPointTestCase;
 import org.apache.lucene.geo.Polygon;
 import org.apache.lucene.geo.Rectangle;
@@ -34,12 +33,12 @@ public class TestGeoPointQuery extends BaseGeoPointTestCase {
   
   @Override
   protected double quantizeLat(double lat) {
-    return GeoEncodingUtils.mortonUnhashLat(GeoEncodingUtils.mortonHash(lat, 0));
+    return GeoPointField.decodeLatitude(GeoPointField.encodeLatLon(lat, 0));
   }
   
   @Override
   protected double quantizeLon(double lon) {
-    return GeoEncodingUtils.mortonUnhashLon(GeoEncodingUtils.mortonHash(0, lon));
+    return GeoPointField.decodeLongitude(GeoPointField.encodeLatLon(0, lon));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/545a1400/lucene/spatial/src/test/org/apache/lucene/spatial/geopoint/search/TestLegacyGeoPointQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/test/org/apache/lucene/spatial/geopoint/search/TestLegacyGeoPointQuery.java b/lucene/spatial/src/test/org/apache/lucene/spatial/geopoint/search/TestLegacyGeoPointQuery.java
index 8766c0e..c2f05ea 100644
--- a/lucene/spatial/src/test/org/apache/lucene/spatial/geopoint/search/TestLegacyGeoPointQuery.java
+++ b/lucene/spatial/src/test/org/apache/lucene/spatial/geopoint/search/TestLegacyGeoPointQuery.java
@@ -18,7 +18,6 @@ package org.apache.lucene.spatial.geopoint.search;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.search.Query;
-import org.apache.lucene.spatial.util.GeoEncodingUtils;
 import org.apache.lucene.geo.BaseGeoPointTestCase;
 import org.apache.lucene.geo.Polygon;
 import org.apache.lucene.geo.Rectangle;
@@ -36,12 +35,12 @@ public class TestLegacyGeoPointQuery extends BaseGeoPointTestCase {
   
   @Override
   protected double quantizeLat(double lat) {
-    return GeoEncodingUtils.mortonUnhashLat(GeoEncodingUtils.mortonHash(lat, 0));
+    return GeoPointField.decodeLatitude(GeoPointField.encodeLatLon(lat, 0));
   }
   
   @Override
   protected double quantizeLon(double lon) {
-    return GeoEncodingUtils.mortonUnhashLon(GeoEncodingUtils.mortonHash(0, lon));
+    return GeoPointField.decodeLongitude(GeoPointField.encodeLatLon(0, lon));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/545a1400/lucene/spatial/src/test/org/apache/lucene/spatial/util/TestGeoEncodingUtils.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/test/org/apache/lucene/spatial/util/TestGeoEncodingUtils.java b/lucene/spatial/src/test/org/apache/lucene/spatial/util/TestGeoEncodingUtils.java
deleted file mode 100644
index aab0de0..0000000
--- a/lucene/spatial/src/test/org/apache/lucene/spatial/util/TestGeoEncodingUtils.java
+++ /dev/null
@@ -1,102 +0,0 @@
-package org.apache.lucene.spatial.util;
-
-/*
- * 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.
- */
-
-import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.lucene.util.LuceneTestCase;
-
-import static org.apache.lucene.geo.GeoTestUtil.nextLatitude;
-import static org.apache.lucene.geo.GeoTestUtil.nextLongitude;
-
-/**
- * Tests methods in {@link GeoEncodingUtils}
- */
-public class TestGeoEncodingUtils extends LuceneTestCase {
-  /**
-   * Tests stability of {@link GeoEncodingUtils#geoCodedToPrefixCoded}
-   */
-  public void testGeoPrefixCoding() throws Exception {
-    int numIters = atLeast(1000);
-    long hash;
-    long decodedHash;
-    BytesRefBuilder brb = new BytesRefBuilder();
-    while (numIters-- >= 0) {
-      hash = GeoEncodingUtils.mortonHash(nextLatitude(), nextLongitude());
-      for (int i=32; i<64; ++i) {
-        GeoEncodingUtils.geoCodedToPrefixCoded(hash, i, brb);
-        decodedHash = GeoEncodingUtils.prefixCodedToGeoCoded(brb.get());
-        assertEquals((hash >>> i) << i, decodedHash);
-      }
-    }
-  }
-
-  public void testMortonEncoding() throws Exception {
-    long hash = GeoEncodingUtils.mortonHash(90, 180);
-    assertEquals(180.0, GeoEncodingUtils.mortonUnhashLon(hash), 0);
-    assertEquals(90.0, GeoEncodingUtils.mortonUnhashLat(hash), 0);
-  }
-
-  public void testEncodeDecode() throws Exception {
-    int iters = atLeast(10000);
-    for(int iter=0;iter<iters;iter++) {
-      double lat = nextLatitude();
-      double lon = nextLongitude();
-
-      long enc = GeoEncodingUtils.mortonHash(lat, lon);
-      double latEnc = GeoEncodingUtils.mortonUnhashLat(enc);
-      double lonEnc = GeoEncodingUtils.mortonUnhashLon(enc);
-
-      // todo remove tolerance
-      assertEquals("lat=" + lat + " latEnc=" + latEnc + " diff=" + (lat - latEnc), lat, latEnc, 1e-6);
-      assertEquals("lon=" + lon + " lonEnc=" + lonEnc + " diff=" + (lon - lonEnc), lon, lonEnc, 1e-6);
-    }
-  }
-
-  /** make sure values always go down: this is important for edge case consistency */
-  public void testEncodeDecodeRoundsDown() throws Exception {
-    int iters = atLeast(1000);
-    for(int iter=0;iter<iters;iter++) {
-      double lat = -90 + 180.0 * random().nextDouble();
-      double lon = -180 + 360.0 * random().nextDouble();
-
-      long enc = GeoEncodingUtils.mortonHash(lat, lon);
-      double latEnc = GeoEncodingUtils.mortonUnhashLat(enc);
-      double lonEnc = GeoEncodingUtils.mortonUnhashLon(enc);
-      assertTrue(latEnc <= lat);
-      assertTrue(lonEnc <= lon);
-    }
-  }
-
-  public void testScaleUnscaleIsStable() throws Exception {
-    int iters = atLeast(1000);
-    for(int iter=0;iter<iters;iter++) {
-      double lat = nextLatitude();
-      double lon = nextLongitude();
-
-      long enc = GeoEncodingUtils.mortonHash(lat, lon);
-      double latEnc = GeoEncodingUtils.mortonUnhashLat(enc);
-      double lonEnc = GeoEncodingUtils.mortonUnhashLon(enc);
-
-      long enc2 = GeoEncodingUtils.mortonHash(lat, lon);
-      double latEnc2 = GeoEncodingUtils.mortonUnhashLat(enc2);
-      double lonEnc2 = GeoEncodingUtils.mortonUnhashLon(enc2);
-      assertEquals(latEnc, latEnc2, 0.0);
-      assertEquals(lonEnc, lonEnc2, 0.0);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/545a1400/lucene/spatial/src/test/org/apache/lucene/spatial/util/TestGeoPointField.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/test/org/apache/lucene/spatial/util/TestGeoPointField.java b/lucene/spatial/src/test/org/apache/lucene/spatial/util/TestGeoPointField.java
new file mode 100644
index 0000000..567d46d
--- /dev/null
+++ b/lucene/spatial/src/test/org/apache/lucene/spatial/util/TestGeoPointField.java
@@ -0,0 +1,51 @@
+package org.apache.lucene.spatial.util;
+
+/*
+ * 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.
+ */
+
+import org.apache.lucene.spatial.geopoint.document.GeoPointField;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.LuceneTestCase;
+
+import static org.apache.lucene.geo.GeoTestUtil.nextLatitude;
+import static org.apache.lucene.geo.GeoTestUtil.nextLongitude;
+import static org.apache.lucene.spatial.geopoint.document.GeoPointField.encodeLatLon;
+import static org.apache.lucene.spatial.geopoint.document.GeoPointField.geoCodedToPrefixCoded;
+import static org.apache.lucene.spatial.geopoint.document.GeoPointField.prefixCodedToGeoCoded;
+
+/**
+ * Tests encoding methods in {@link GeoPointField}
+ */
+public class TestGeoPointField extends LuceneTestCase {
+  /**
+   * Tests stability of {@link GeoPointField#geoCodedToPrefixCoded}
+   */
+  public void testGeoPrefixCoding() throws Exception {
+    int numIters = atLeast(1000);
+    long hash;
+    long decodedHash;
+    BytesRefBuilder brb = new BytesRefBuilder();
+    while (numIters-- >= 0) {
+      hash = encodeLatLon(nextLatitude(), nextLongitude());
+      for (int i=32; i<64; ++i) {
+        geoCodedToPrefixCoded(hash, i, brb);
+        decodedHash = prefixCodedToGeoCoded(brb.get());
+        assertEquals((hash >>> i) << i, decodedHash);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/545a1400/lucene/spatial/src/test/org/apache/lucene/spatial/util/TestMortonEncoder.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/test/org/apache/lucene/spatial/util/TestMortonEncoder.java b/lucene/spatial/src/test/org/apache/lucene/spatial/util/TestMortonEncoder.java
new file mode 100644
index 0000000..9096e6e
--- /dev/null
+++ b/lucene/spatial/src/test/org/apache/lucene/spatial/util/TestMortonEncoder.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.lucene.spatial.util;
+
+import java.util.Random;
+
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.NumericUtils;
+import org.apache.lucene.util.TestUtil;
+
+import static org.apache.lucene.geo.GeoEncodingUtils.decodeLatitude;
+import static org.apache.lucene.geo.GeoEncodingUtils.decodeLongitude;
+import static org.apache.lucene.spatial.util.MortonEncoder.decodeLatitude;
+import static org.apache.lucene.spatial.util.MortonEncoder.decodeLongitude;
+import static org.apache.lucene.spatial.util.MortonEncoder.encode;
+import static org.apache.lucene.spatial.util.MortonEncoder.encodeCeil;
+
+import static org.apache.lucene.util.BitUtil.deinterleave;
+import static org.apache.lucene.util.BitUtil.interleave;
+
+/**
+ * Tests methods in {@link MortonEncoder}
+ */
+public class TestMortonEncoder extends LuceneTestCase {
+
+  public void testMortonEncoding() throws Exception {
+    final long TRANSLATE = 1L << 31;
+    final double LATITUDE_DECODE = 180.0D/(0x1L<<32);
+    final double LONGITUDE_DECODE = 360.0D/(0x1L<<32);
+    Random random = random();
+    for(int i=0; i < 10000; ++i) {
+      long encoded = random().nextLong();
+      long encodedLat = deinterleave(encoded >>> 1);
+      long encodedLon = deinterleave(encoded);
+      double expectedLat = decodeLatitude((int)(encodedLat - TRANSLATE));
+      double decodedLat = decodeLatitude(encoded);
+      double expectedLon = decodeLongitude((int)(encodedLon - TRANSLATE));
+      double decodedLon = decodeLongitude(encoded);
+      assertEquals(expectedLat, decodedLat, 0.0D);
+      assertEquals(expectedLon, decodedLon, 0.0D);
+      // should round-trip
+      assertEquals(encoded, encode(decodedLat, decodedLon));
+
+      // test within the range
+      if (encoded != 0xFFFFFFFFFFFFFFFFL) {
+        // this is the next representable value
+        // all double values between [min .. max) should encode to the current integer
+        // all double values between (min .. max] should encodeCeil to the next integer.
+        double maxLat = expectedLat + LATITUDE_DECODE;
+        encodedLat += 1;
+        assertEquals(maxLat, decodeLatitude((int)(encodedLat - TRANSLATE)), 0.0D);
+        double maxLon = expectedLon + LONGITUDE_DECODE;
+        encodedLon += 1;
+        assertEquals(maxLon, decodeLongitude((int)(encodedLon - TRANSLATE)), 0.0D);
+        long encodedNext = encode(maxLat, maxLon);
+        assertEquals(interleave((int)encodedLon, (int)encodedLat), encodedNext);
+
+        // first and last doubles in range that will be quantized
+        double minEdgeLat = Math.nextUp(expectedLat);
+        double minEdgeLon = Math.nextUp(expectedLon);
+        long encodedMinEdge = encode(minEdgeLat, minEdgeLon);
+        long encodedMinEdgeCeil = encodeCeil(minEdgeLat, minEdgeLon);
+        double maxEdgeLat = Math.nextDown(maxLat);
+        double maxEdgeLon = Math.nextDown(maxLon);
+        long encodedMaxEdge = encode(maxEdgeLat, maxEdgeLon);
+        long encodedMaxEdgeCeil = encodeCeil(maxEdgeLat, maxEdgeLon);
+
+        assertEquals(encodedLat - 1, deinterleave(encodedMinEdge >>> 1));
+        assertEquals(encodedLat, deinterleave(encodedMinEdgeCeil >>> 1));
+        assertEquals(encodedLon - 1, deinterleave(encodedMinEdge));
+        assertEquals(encodedLon, deinterleave(encodedMinEdgeCeil));
+
+        assertEquals(encodedLat - 1, deinterleave(encodedMaxEdge >>> 1));
+        assertEquals(encodedLat, deinterleave(encodedMaxEdgeCeil >>> 1));
+        assertEquals(encodedLon - 1, deinterleave(encodedMaxEdge));
+        assertEquals(encodedLon, deinterleave(encodedMaxEdgeCeil));
+
+        // check random values within the double range
+        long minBitsLat = NumericUtils.doubleToSortableLong(minEdgeLat);
+        long maxBitsLat = NumericUtils.doubleToSortableLong(maxEdgeLat);
+        long minBitsLon = NumericUtils.doubleToSortableLong(minEdgeLon);
+        long maxBitsLon = NumericUtils.doubleToSortableLong(maxEdgeLon);
+        for (int j = 0; j < 100; j++) {
+          double valueLat = NumericUtils.sortableLongToDouble(TestUtil.nextLong(random, minBitsLat, maxBitsLat));
+          double valueLon = NumericUtils.sortableLongToDouble(TestUtil.nextLong(random, minBitsLon, maxBitsLon));
+          // round down
+          assertEquals(encoded,   encode(valueLat, valueLon));
+          // round up
+          assertEquals(interleave((int)encodedLon, (int)encodedLat), encodeCeil(valueLat, valueLon));
+        }
+      }
+    }
+  }
+}


[49/50] [abbrv] lucene-solr:solr-5750: Merge remote-tracking branch 'origin/master' into solr-5750

Posted by ds...@apache.org.
Merge remote-tracking branch 'origin/master' into solr-5750


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

Branch: refs/heads/solr-5750
Commit: e54718df7166a54b2dc44909bd5cffca4ba4fcd6
Parents: cf77521 3b4ec73
Author: David Smiley <ds...@apache.org>
Authored: Wed Apr 27 11:47:14 2016 -0400
Committer: David Smiley <ds...@apache.org>
Committed: Wed Apr 27 11:47:14 2016 -0400

----------------------------------------------------------------------
 .gitignore                                      |    2 +
 dev-tools/idea/.idea/libraries/Ant.xml          |    9 -
 dev-tools/idea/.idea/libraries/JUnit.xml        |    4 +-
 dev-tools/idea/.idea/modules.xml                |    1 +
 dev-tools/idea/.idea/workspace.xml              |   13 +-
 .../lucene/classification/classification.iml    |    2 +
 dev-tools/idea/lucene/tools/tools.iml           |    1 -
 .../idea/solr/contrib/analytics/analytics.iml   |    1 -
 dev-tools/idea/solr/server/server.iml           |   19 +
 dev-tools/maven/lucene/sandbox/pom.xml.template |    7 -
 dev-tools/maven/solr/pom.xml.template           |    9 +
 dev-tools/scripts/addVersion.py                 |   19 +-
 dev-tools/scripts/buildAndPushRelease.py        |    6 +-
 dev-tools/scripts/scriptutil.py                 |   20 +-
 lucene/CHANGES.txt                              |  142 +-
 .../DictionaryCompoundWordTokenFilter.java      |    4 +-
 .../lucene/analysis/hunspell/Dictionary.java    |   19 +-
 .../payloads/NumericPayloadTokenFilter.java     |    2 +-
 .../index/TestBackwardsCompatibility.java       |    5 +-
 .../org/apache/lucene/index/index.6.0.0-cfs.zip |  Bin 0 -> 13744 bytes
 .../apache/lucene/index/index.6.0.0-nocfs.zip   |  Bin 0 -> 13749 bytes
 lucene/classification/build.xml                 |    8 +-
 .../utils/ConfusionMatrixGenerator.java         |   80 +-
 .../classification/utils/DatasetSplitter.java   |  126 +-
 .../BooleanPerceptronClassifierTest.java        |    2 +-
 .../CachingNaiveBayesClassifierTest.java        |    2 +-
 .../KNearestNeighborClassifierTest.java         |    2 +-
 .../SimpleNaiveBayesClassifierTest.java         |    2 +-
 .../utils/ConfusionMatrixGeneratorTest.java     |  105 +-
 .../classification/utils/DataSplitterTest.java  |   33 +-
 .../simpletext/SimpleTextPointsWriter.java      |    5 +-
 lucene/common-build.xml                         |    4 +-
 .../org/apache/lucene/codecs/CodecUtil.java     |    4 +-
 .../org/apache/lucene/codecs/PointsWriter.java  |    5 +-
 .../codecs/lucene60/Lucene60PointsReader.java   |    5 +-
 .../codecs/lucene60/Lucene60PointsWriter.java   |   21 +-
 .../org/apache/lucene/codecs/package-info.java  |    1 +
 .../org/apache/lucene/document/BinaryPoint.java |   21 +-
 .../org/apache/lucene/document/DoublePoint.java |   19 +-
 .../java/org/apache/lucene/document/Field.java  |   26 +-
 .../org/apache/lucene/document/FieldType.java   |    4 +-
 .../org/apache/lucene/document/FloatPoint.java  |   19 +-
 .../org/apache/lucene/document/IntPoint.java    |   19 +-
 .../org/apache/lucene/document/LongPoint.java   |   19 +-
 .../org/apache/lucene/document/StringField.java |    4 +-
 .../org/apache/lucene/geo/GeoEncodingUtils.java |  147 ++
 .../java/org/apache/lucene/geo/GeoUtils.java    |   94 ++
 .../src/java/org/apache/lucene/geo/Polygon.java |  162 ++
 .../java/org/apache/lucene/geo/Polygon2D.java   |  473 ++++++
 .../java/org/apache/lucene/geo/Rectangle.java   |  189 +++
 .../org/apache/lucene/geo/package-info.java     |   21 +
 .../lucene/index/DefaultIndexingChain.java      |    2 +-
 .../lucene/index/DocValuesFieldUpdates.java     |    2 +-
 .../java/org/apache/lucene/index/FieldInfo.java |    8 +-
 .../org/apache/lucene/index/FieldInfos.java     |    2 +-
 .../apache/lucene/index/FilterLeafReader.java   |   10 +-
 .../org/apache/lucene/index/IndexWriter.java    |    2 +-
 .../apache/lucene/index/IndexWriterConfig.java  |    6 +-
 .../org/apache/lucene/index/PointValues.java    |   38 +-
 .../apache/lucene/index/PointValuesWriter.java  |   10 +-
 .../org/apache/lucene/search/LRUQueryCache.java |  303 +++-
 .../apache/lucene/search/PointInSetQuery.java   |    4 +-
 .../apache/lucene/search/PointRangeQuery.java   |   98 +-
 .../org/apache/lucene/search/PrefixQuery.java   |    2 +-
 .../lucene/search/QueryCachingPolicy.java       |   69 +-
 .../apache/lucene/search/ReferenceManager.java  |    4 +-
 .../search/UsageTrackingQueryCachingPolicy.java |   29 +-
 .../org/apache/lucene/store/BaseDirectory.java  |    2 +-
 .../org/apache/lucene/store/NIOFSDirectory.java |    2 +-
 .../lucene/store/NRTCachingDirectory.java       |    5 +-
 .../apache/lucene/store/SimpleFSDirectory.java  |    2 +-
 .../java/org/apache/lucene/util/BitUtil.java    |    4 +-
 .../org/apache/lucene/util/BytesRefArray.java   |   15 +-
 .../org/apache/lucene/util/CommandLineUtil.java |    2 +-
 .../lucene/util/FixedLengthBytesRefArray.java   |  189 +++
 .../org/apache/lucene/util/OfflineSorter.java   |  164 +-
 .../java/org/apache/lucene/util/SloppyMath.java |  126 +-
 .../lucene/util/SortableBytesRefArray.java      |   31 +
 .../org/apache/lucene/util/UnicodeUtil.java     |    2 +-
 .../org/apache/lucene/util/bkd/BKDReader.java   |   47 +-
 .../org/apache/lucene/util/bkd/BKDWriter.java   |  235 +--
 .../apache/lucene/util/bkd/HeapPointReader.java |    8 +-
 .../apache/lucene/util/bkd/HeapPointWriter.java |   64 +-
 .../lucene/util/bkd/OfflinePointReader.java     |  119 +-
 .../lucene/util/bkd/OfflinePointWriter.java     |   82 +-
 .../org/apache/lucene/util/bkd/PointReader.java |   12 +
 .../org/apache/lucene/util/bkd/PointWriter.java |    6 +-
 .../apache/lucene/geo/TestGeoEncodingUtils.java |  151 ++
 .../org/apache/lucene/geo/TestGeoUtils.java     |  293 ++++
 .../test/org/apache/lucene/geo/TestPolygon.java |   62 +
 .../org/apache/lucene/geo/TestPolygon2D.java    |  289 ++++
 .../lucene/index/Test4GBStoredFields.java       |   24 +-
 .../apache/lucene/index/TestPointValues.java    |   50 +
 .../apache/lucene/search/TestIndexSearcher.java |    2 +-
 .../apache/lucene/search/TestLRUQueryCache.java |   79 +-
 .../apache/lucene/search/TestPointQueries.java  |  161 +-
 .../lucene/search/TestQueryCachingPolicy.java   |   56 -
 .../TestUsageTrackingFilterCachingPolicy.java   |   17 +-
 .../search/spans/TestFieldMaskingSpanQuery.java |    2 +-
 .../apache/lucene/store/TestRateLimiter.java    |    2 +-
 .../util/TestFixedLengthBytesRefArray.java      |   84 +
 .../apache/lucene/util/TestOfflineSorter.java   |  265 ++-
 .../org/apache/lucene/util/TestSloppyMath.java  |  137 +-
 .../apache/lucene/util/bkd/Test2BBKDPoints.java |   21 +-
 .../org/apache/lucene/util/bkd/TestBKD.java     |  267 ++-
 .../demo/facet/DistanceFacetsExample.java       |    8 +-
 lucene/demo/src/java/overview.html              |    2 +-
 .../js/JavascriptCompiler.properties            |    2 +-
 .../lucene/expressions/TestDemoExpressions.java |    6 +-
 .../expressions/TestExpressionValueSource.java  |    4 +-
 .../expressions/js/TestJavascriptFunction.java  |    2 +-
 .../org/apache/lucene/facet/range/Range.java    |    2 +-
 .../directory/DirectoryTaxonomyReader.java      |    2 +-
 .../facet/range/TestRangeFacetCounts.java       |   13 +-
 .../highlight/WeightedSpanTermExtractor.java    |    7 +-
 .../postingshighlight/PostingsHighlighter.java  |    4 +-
 .../custom/HighlightCustomQueryTest.java        |   15 +
 lucene/ivy-versions.properties                  |    2 +-
 .../org/apache/lucene/search/join/JoinUtil.java |  274 ++-
 .../join/PointInSetIncludingScoreQuery.java     |  340 ++++
 .../search/join/ToChildBlockJoinQuery.java      |    4 +-
 .../search/join/ToParentBlockJoinQuery.java     |    6 +-
 .../lucene/search/join/TestBlockJoin.java       |   32 +-
 .../apache/lucene/search/join/TestJoinUtil.java |   54 +-
 .../randomizedtesting-runner-2.3.2.jar.sha1     |    1 -
 .../randomizedtesting-runner-2.3.4.jar.sha1     |    1 +
 .../apache/lucene/index/memory/MemoryIndex.java |    1 +
 .../apache/lucene/store/WindowsDirectory.cpp    |    4 +-
 .../apache/lucene/uninverting/DocTermOrds.java  |    6 +-
 .../apache/lucene/uninverting/FieldCache.java   |    2 +-
 .../lucene/queries/function/FunctionQuery.java  |   19 +-
 .../queries/function/FunctionRangeQuery.java    |    5 +-
 .../lucene/queries/function/FunctionValues.java |   20 +-
 .../queries/function/ValueSourceScorer.java     |    7 +-
 .../docvalues/DocTermsIndexDocValues.java       |    5 +-
 .../function/docvalues/DoubleDocValues.java     |   16 +-
 .../function/docvalues/IntDocValues.java        |    9 +-
 .../function/docvalues/LongDocValues.java       |    9 +-
 .../function/valuesource/EnumFieldSource.java   |    8 +-
 .../apache/lucene/queries/mlt/MoreLikeThis.java |  129 +-
 .../apache/lucene/queries/TermsQueryTest.java   |   16 +-
 .../queries/TestCustomScoreExplanations.java    |   46 +
 .../lucene/queries/mlt/TestMoreLikeThis.java    |   82 +
 .../queryparser/classic/QueryParserBase.java    |    4 +-
 .../core/config/AbstractQueryConfig.java        |    8 +-
 .../flexible/core/config/FieldConfig.java       |    4 +-
 .../standard/config/LegacyNumericConfig.java    |    6 +-
 .../LegacyNumericFieldConfigListener.java       |    2 +-
 .../flexible/standard/config/PointsConfig.java  |    6 +-
 .../standard/config/PointsConfigListener.java   |    2 +-
 .../nodes/LegacyNumericRangeQueryNode.java      |    2 +-
 .../standard/nodes/PointRangeQueryNode.java     |    2 +-
 .../lucene/queryparser/xml/CoreParser.java      |    2 +-
 .../xml/builders/BooleanQueryBuilder.java       |    3 +-
 .../builders/DisjunctionMaxQueryBuilder.java    |    3 +-
 .../xml/builders/FuzzyLikeThisQueryBuilder.java |    3 +-
 .../LegacyNumericRangeQueryBuilder.java         |   36 +-
 .../xml/builders/PointRangeQueryBuilder.java    |   28 +-
 .../flexible/standard/TestPointQueryParser.java |    2 +-
 .../xml/CoreParserTestIndexData.java            |   74 +
 .../LegacyNumericRangeQueryWithoutLowerTerm.xml |   31 +
 .../xml/LegacyNumericRangeQueryWithoutRange.xml |   31 +
 .../LegacyNumericRangeQueryWithoutUpperTerm.xml |   31 +
 .../xml/PointRangeQueryWithoutLowerTerm.xml     |   31 +
 .../xml/PointRangeQueryWithoutRange.xml         |   31 +
 .../xml/PointRangeQueryWithoutUpperTerm.xml     |   31 +
 .../lucene/queryparser/xml/TermQueryEmpty.xml   |   18 +
 .../lucene/queryparser/xml/TestCoreParser.java  |  159 +-
 .../xml/TestCorePlusExtensionsParser.java       |   18 +-
 .../xml/TestCorePlusQueriesParser.java          |   18 +-
 .../apache/lucene/replicator/RevisionFile.java  |    2 +-
 .../replicator/http/ReplicationService.java     |    3 +-
 lucene/sandbox/build.xml                        |   25 -
 .../apache/lucene/document/BigIntegerPoint.java |   16 +-
 .../lucene/document/InetAddressPoint.java       |   69 +-
 .../lucene/document/LatLonDocValuesField.java   |  135 ++
 .../org/apache/lucene/document/LatLonPoint.java |  304 ++--
 .../lucene/document/LatLonPointBoxQuery.java    |  287 ++++
 .../document/LatLonPointDistanceComparator.java |   54 +-
 .../document/LatLonPointDistanceQuery.java      |  200 +--
 .../document/LatLonPointInPolygonQuery.java     |  258 +--
 .../lucene/document/LatLonPointSortField.java   |   12 +-
 .../apache/lucene/document/MatchingPoints.java  |   90 +
 .../apache/lucene/document/NearestNeighbor.java |  308 ++++
 .../lucene/search/DocValuesRangeQuery.java      |    8 +-
 .../lucene/document/TestBigIntegerPoint.java    |   26 +-
 .../lucene/document/TestInetAddressPoint.java   |   96 +-
 .../document/TestLatLonDocValuesField.java      |   30 +
 .../apache/lucene/document/TestLatLonPoint.java |  181 +-
 .../document/TestLatLonPointDistanceQuery.java  |  190 ---
 .../document/TestLatLonPointDistanceSort.java   |   95 +-
 .../document/TestLatLonPointInPolygonQuery.java |   49 -
 .../org/apache/lucene/document/TestNearest.java |  253 +++
 .../lucene/search/TestDocValuesRangeQuery.java  |   34 +
 .../lucene/search/TestLatLonPointQueries.java   |  195 +--
 .../lucene/spatial/bbox/BBoxStrategy.java       |  318 ++--
 .../prefix/IntersectsPrefixTreeQuery.java       |   13 +-
 .../spatial/vector/PointVectorStrategy.java     |  177 +-
 .../lucene/spatial/DistanceStrategyTest.java    |   40 +-
 .../apache/lucene/spatial/PortedSolr3Test.java  |   13 +-
 .../lucene/spatial/QueryEqualsHashCodeTest.java |   10 +-
 .../apache/lucene/spatial/SpatialTestCase.java  |   42 +-
 .../lucene/spatial/bbox/TestBBoxStrategy.java   |   68 +-
 .../composite/CompositeStrategyTest.java        |   17 +-
 .../serialized/SerializedStrategyTest.java      |    7 +-
 .../lucene/spatial/spatial4j/Geo3dRptTest.java  |   53 +-
 .../Geo3dShapeRectRelationTestCase.java         |   28 +-
 .../Geo3dShapeSphereModelRectRelationTest.java  |   10 +-
 .../Geo3dShapeWGS84ModelRectRelationTest.java   |   19 +-
 .../spatial/vector/TestPointVectorStrategy.java |   55 +-
 .../geopoint/document/GeoPointField.java        |  148 +-
 .../geopoint/document/GeoPointTokenStream.java  |   14 +-
 .../spatial/geopoint/search/GeoBoundingBox.java |   53 -
 .../geopoint/search/GeoPointDistanceQuery.java  |   84 +-
 .../search/GeoPointDistanceQueryImpl.java       |   79 +-
 .../search/GeoPointDistanceRangeQuery.java      |  122 --
 .../geopoint/search/GeoPointInBBoxQuery.java    |   82 +-
 .../search/GeoPointInBBoxQueryImpl.java         |   51 +-
 .../geopoint/search/GeoPointInPolygonQuery.java |  132 +-
 .../search/GeoPointInPolygonQueryImpl.java      |   58 +-
 .../geopoint/search/GeoPointMultiTermQuery.java |   56 +-
 .../search/GeoPointNumericTermsEnum.java        |   23 +-
 .../search/GeoPointPrefixTermsEnum.java         |   40 +-
 .../GeoPointTermQueryConstantScoreWrapper.java  |    7 +-
 .../geopoint/search/GeoPointTermsEnum.java      |    4 +-
 .../lucene/spatial/util/GeoDistanceUtils.java   |  223 ---
 .../lucene/spatial/util/GeoEncodingUtils.java   |  157 --
 .../lucene/spatial/util/GeoHashUtils.java       |  283 ----
 .../lucene/spatial/util/GeoProjectionUtils.java |  465 ------
 .../org/apache/lucene/spatial/util/GeoRect.java |   78 -
 .../lucene/spatial/util/GeoRelationUtils.java   |  485 +-----
 .../apache/lucene/spatial/util/GeoUtils.java    |  187 ---
 .../lucene/spatial/util/MortonEncoder.java      |  103 ++
 .../geopoint/search/GeoPointTestUtil.java       |  287 ++++
 .../geopoint/search/TestGeoPointQuery.java      |  368 +---
 .../search/TestLegacyGeoPointQuery.java         |  103 ++
 .../spatial/util/BaseGeoPointTestCase.java      |  823 ---------
 .../lucene/spatial/util/TestGeoPointField.java  |   51 +
 .../lucene/spatial/util/TestGeoUtils.java       |  565 -------
 .../lucene/spatial/util/TestMortonEncoder.java  |  108 ++
 .../org/apache/lucene/spatial3d/Geo3DPoint.java |  183 +-
 .../org/apache/lucene/spatial3d/Geo3DUtil.java  |   65 +-
 .../spatial3d/PointInGeo3DShapeQuery.java       |   83 +-
 .../spatial3d/PointInShapeIntersectVisitor.java |  117 ++
 .../lucene/spatial3d/geom/ArcDistance.java      |   15 +
 .../lucene/spatial3d/geom/BaseXYZSolid.java     |    2 +-
 .../lucene/spatial3d/geom/DistanceStyle.java    |   28 +
 .../lucene/spatial3d/geom/GeoBBoxFactory.java   |    3 +-
 .../lucene/spatial3d/geom/GeoBaseBBox.java      |    2 +-
 .../lucene/spatial3d/geom/GeoBaseCircle.java    |    4 +-
 .../spatial3d/geom/GeoBaseDistanceShape.java    |   12 +
 .../lucene/spatial3d/geom/GeoBasePath.java      |   34 +
 .../lucene/spatial3d/geom/GeoBasePolygon.java   |    4 +-
 .../spatial3d/geom/GeoConcavePolygon.java       |  471 ++++++
 .../lucene/spatial3d/geom/GeoConvexPolygon.java |  293 +++-
 .../geom/GeoDegenerateHorizontalLine.java       |    2 +-
 .../geom/GeoDegenerateLatitudeZone.java         |    2 +-
 .../geom/GeoDegenerateLongitudeSlice.java       |    2 +-
 .../spatial3d/geom/GeoDegeneratePoint.java      |    8 +-
 .../lucene/spatial3d/geom/GeoDistance.java      |    1 +
 .../lucene/spatial3d/geom/GeoDistanceShape.java |   12 +
 .../lucene/spatial3d/geom/GeoLatitudeZone.java  |    2 +-
 .../spatial3d/geom/GeoLongitudeSlice.java       |    2 +-
 .../spatial3d/geom/GeoNorthLatitudeZone.java    |    2 +-
 .../spatial3d/geom/GeoNorthRectangle.java       |    2 +-
 .../apache/lucene/spatial3d/geom/GeoPath.java   |  776 +--------
 .../lucene/spatial3d/geom/GeoPathFactory.java   |   39 +
 .../apache/lucene/spatial3d/geom/GeoPoint.java  |    2 +-
 .../spatial3d/geom/GeoPolygonFactory.java       | 1565 ++++++++++++++++--
 .../lucene/spatial3d/geom/GeoRectangle.java     |    2 +-
 .../spatial3d/geom/GeoSouthLatitudeZone.java    |    2 +-
 .../spatial3d/geom/GeoSouthRectangle.java       |    2 +-
 .../spatial3d/geom/GeoStandardCircle.java       |    8 +-
 .../lucene/spatial3d/geom/GeoStandardPath.java  |  803 +++++++++
 .../geom/GeoWideDegenerateHorizontalLine.java   |    2 +-
 .../spatial3d/geom/GeoWideLongitudeSlice.java   |    2 +-
 .../spatial3d/geom/GeoWideNorthRectangle.java   |    2 +-
 .../lucene/spatial3d/geom/GeoWideRectangle.java |    2 +-
 .../spatial3d/geom/GeoWideSouthRectangle.java   |    2 +-
 .../apache/lucene/spatial3d/geom/GeoWorld.java  |    2 +-
 .../lucene/spatial3d/geom/LatLonBounds.java     |   18 +-
 .../lucene/spatial3d/geom/LinearDistance.java   |   15 +
 .../spatial3d/geom/LinearSquaredDistance.java   |   15 +
 .../lucene/spatial3d/geom/NormalDistance.java   |   15 +
 .../spatial3d/geom/NormalSquaredDistance.java   |   15 +
 .../org/apache/lucene/spatial3d/geom/Plane.java |  214 ++-
 .../lucene/spatial3d/geom/PlanetModel.java      |  148 +-
 .../lucene/spatial3d/geom/SidedPlane.java       |   20 +-
 .../lucene/spatial3d/geom/StandardXYZSolid.java |   59 +-
 .../apache/lucene/spatial3d/geom/Vector.java    |   16 +-
 .../apache/lucene/spatial3d/geom/XYZBounds.java |   52 +-
 .../apache/lucene/spatial3d/geom/XYdZSolid.java |    2 +-
 .../apache/lucene/spatial3d/geom/XdYZSolid.java |    2 +-
 .../lucene/spatial3d/geom/XdYdZSolid.java       |    2 +-
 .../apache/lucene/spatial3d/geom/dXYZSolid.java |    2 +-
 .../lucene/spatial3d/geom/dXYdZSolid.java       |    2 +-
 .../lucene/spatial3d/geom/dXdYZSolid.java       |    2 +-
 .../lucene/spatial3d/geom/dXdYdZSolid.java      |    2 +-
 .../apache/lucene/spatial3d/TestGeo3DPoint.java | 1028 +++++++++---
 .../lucene/spatial3d/geom/GeoBBoxTest.java      |    8 +-
 .../lucene/spatial3d/geom/GeoCircleTest.java    |   32 +-
 .../lucene/spatial3d/geom/GeoPathTest.java      |   36 +-
 .../lucene/spatial3d/geom/GeoPointTest.java     |   33 +
 .../lucene/spatial3d/geom/GeoPolygonTest.java   |  383 ++++-
 .../apache/lucene/spatial3d/geom/PlaneTest.java |   20 +
 .../search/suggest/SortedInputIterator.java     |   33 +-
 .../suggest/analyzing/AnalyzingSuggester.java   |   26 +-
 .../search/suggest/fst/ExternalRefSorter.java   |   23 +-
 .../search/suggest/fst/FSTCompletionLookup.java |   22 +-
 .../search/suggest/fst/BytesRefSortersTest.java |   11 +-
 .../codecs/compressing/CompressingCodec.java    |   13 +
 .../apache/lucene/geo/BaseGeoPointTestCase.java | 1489 +++++++++++++++++
 .../org/apache/lucene/geo/EarthDebugger.java    |  293 ++++
 .../java/org/apache/lucene/geo/GeoTestUtil.java |  699 ++++++++
 .../src/java/org/apache/lucene/geo/package.html |   26 +
 .../lucene/index/BasePointsFormatTestCase.java  |    2 +-
 .../apache/lucene/index/PointsStackTracker.java |   79 +
 .../org/apache/lucene/index/RandomCodec.java    |   92 +-
 .../lucene/store/CorruptingIndexOutput.java     |  101 ++
 .../org/apache/lucene/util/LuceneTestCase.java  |   13 +-
 solr/CHANGES.txt                                |  205 ++-
 solr/bin/post                                   |    9 +-
 solr/common-build.xml                           |    7 +-
 .../analytics/accumulator/BasicAccumulator.java |    6 +-
 .../accumulator/FacetingAccumulator.java        |    8 +-
 .../solr/analytics/expression/Expression.java   |   14 +-
 .../analytics/expression/ExpressionFactory.java |    9 +-
 .../StatsCollectorSupplierFactory.java          |   21 +-
 .../solr/analytics/util/AnalyticsParsers.java   |    7 +-
 .../analytics/util/RangeEndpointCalculator.java |    5 +-
 .../util/valuesource/ConstDateSource.java       |    9 +-
 .../util/valuesource/DateFieldSource.java       |    6 +-
 .../util/valuesource/FilterFieldSource.java     |    3 +-
 .../analytics/expression/ExpressionTest.java    |   27 +-
 .../handler/dataimport/SolrEntityProcessor.java |   10 +-
 .../dataimport/TestContentStreamDataSource.java |    4 +-
 .../TestSolrEntityProcessorEndToEnd.java        |    2 +-
 .../extraction/ExtractingRequestHandler.java    |   32 +-
 .../handler/extraction/ExtractionDateUtil.java  |  178 ++
 .../handler/extraction/SolrContentHandler.java  |   12 +-
 .../extraction/TestExtractionDateUtil.java      |   61 +
 .../src/java/org/apache/solr/hadoop/GoLive.java |   31 +-
 .../solr/hadoop/MapReduceIndexerTool.java       |    7 +-
 .../apache/solr/hadoop/ZooKeeperInspector.java  |    9 +-
 .../test/org/apache/solr/hadoop/MRUnitBase.java |    2 +
 .../MapReduceIndexerToolArgumentParserTest.java |    1 +
 .../solr/hadoop/MorphlineBasicMiniMRTest.java   |    1 +
 .../solr/hadoop/MorphlineGoLiveMiniMRTest.java  |    4 +-
 .../solr/morphlines/cell/SolrCellBuilder.java   |   18 +-
 .../morphlines/cell/SolrCellMorphlineTest.java  |    7 +-
 .../solr/SafeConcurrentUpdateSolrClient.java    |    2 +-
 .../solr/morphlines/solr/SolrLocator.java       |    5 +-
 .../solr/AbstractSolrMorphlineTestBase.java     |    5 +-
 .../solr/AbstractSolrMorphlineZkTestBase.java   |    3 +
 .../solr/SolrMorphlineZkAvroTest.java           |   14 +-
 .../morphlines/solr/SolrMorphlineZkTest.java    |    5 -
 .../client/solrj/embedded/JettySolrRunner.java  |   25 +-
 .../src/java/org/apache/solr/cloud/Assign.java  |    8 +-
 .../org/apache/solr/cloud/LeaderElector.java    |    5 +-
 .../cloud/LeaderInitiatedRecoveryThread.java    |    2 +-
 .../java/org/apache/solr/cloud/Overseer.java    |    1 +
 .../OverseerAutoReplicaFailoverThread.java      |   41 +-
 .../cloud/OverseerCollectionMessageHandler.java |   65 +-
 .../cloud/OverseerConfigSetMessageHandler.java  |    1 +
 .../apache/solr/cloud/OverseerTaskQueue.java    |   21 +-
 .../org/apache/solr/cloud/RecoveryStrategy.java |   11 +-
 .../org/apache/solr/cloud/SyncStrategy.java     |    2 +-
 .../src/java/org/apache/solr/cloud/ZkCLI.java   |   14 +-
 .../org/apache/solr/cloud/ZkController.java     |   86 +-
 .../apache/solr/cloud/ZkSolrResourceLoader.java |   41 +-
 .../cloud/overseer/ClusterStateMutator.java     |   16 +-
 .../solr/cloud/overseer/CollectionMutator.java  |    6 +-
 .../apache/solr/cloud/overseer/NodeMutator.java |    9 +-
 .../solr/cloud/overseer/ReplicaMutator.java     |   36 +-
 .../solr/cloud/overseer/SliceMutator.java       |   39 +-
 .../solr/cloud/overseer/ZkStateWriter.java      |    3 +-
 .../apache/solr/cloud/rule/RemoteCallback.java  |    2 +-
 .../apache/solr/cloud/rule/SnitchContext.java   |    3 +-
 .../org/apache/solr/core/BlobRepository.java    |  221 +--
 .../solr/core/CachingDirectoryFactory.java      |   20 +-
 .../org/apache/solr/core/ConfigSetService.java  |    9 +-
 .../org/apache/solr/core/CoreContainer.java     |  113 +-
 .../org/apache/solr/core/DirectoryFactory.java  |    6 +
 .../apache/solr/core/HdfsDirectoryFactory.java  |   33 +-
 .../org/apache/solr/core/JmxMonitoredMap.java   |   23 +-
 .../java/org/apache/solr/core/PluginBag.java    |   35 +-
 .../solr/core/ShutdownAwareDirectory.java       |   30 +
 .../src/java/org/apache/solr/core/SolrCore.java |  155 +-
 .../java/org/apache/solr/core/SolrCores.java    |   25 +-
 .../apache/solr/core/SolrDeletionPolicy.java    |   16 +-
 .../apache/solr/core/SolrInfoMBeanWrapper.java  |   62 +
 .../solr/handler/CdcrReplicatorManager.java     |   18 +-
 .../solr/handler/CdcrReplicatorScheduler.java   |   19 +-
 .../solr/handler/CdcrReplicatorState.java       |   21 +-
 .../apache/solr/handler/CdcrRequestHandler.java |    2 +-
 .../solr/handler/CdcrUpdateLogSynchronizer.java |    6 +-
 .../org/apache/solr/handler/IndexFetcher.java   |   68 +-
 .../solr/handler/MoreLikeThisHandler.java       |   12 +-
 .../solr/handler/NestedRequestHandler.java      |    2 +-
 .../apache/solr/handler/PingRequestHandler.java |    6 +-
 .../apache/solr/handler/ReplicationHandler.java |   25 +-
 .../org/apache/solr/handler/SQLHandler.java     |  161 +-
 .../org/apache/solr/handler/SchemaHandler.java  |   62 +-
 .../apache/solr/handler/SolrConfigHandler.java  |   19 +-
 .../org/apache/solr/handler/StreamHandler.java  |   89 +-
 .../solr/handler/UpdateRequestHandler.java      |    9 +-
 .../solr/handler/admin/CollectionsHandler.java  |   45 +-
 .../solr/handler/admin/CoreAdminHandler.java    |   18 +-
 .../solr/handler/admin/CoreAdminOperation.java  |   11 +-
 .../solr/handler/admin/SecurityConfHandler.java |   16 +-
 .../solr/handler/component/DebugComponent.java  |    1 +
 .../handler/component/HighlightComponent.java   |    1 +
 .../handler/component/HttpShardHandler.java     |  100 +-
 .../component/HttpShardHandlerFactory.java      |   48 +-
 .../component/IterativeMergeStrategy.java       |   10 +-
 .../component/MoreLikeThisComponent.java        |    3 +
 .../solr/handler/component/PivotFacetValue.java |    3 +-
 .../solr/handler/component/QueryComponent.java  |   13 +-
 .../handler/component/RangeFacetRequest.java    |   50 +-
 .../handler/component/RealTimeGetComponent.java |   13 +-
 .../solr/handler/component/SearchHandler.java   |   11 +-
 .../handler/component/ShardHandlerFactory.java  |    2 +-
 .../request/PerSegmentSingleValuedFaceting.java |    9 +-
 .../org/apache/solr/request/SimpleFacets.java   |   35 +-
 .../solr/response/BinaryResponseWriter.java     |   11 +-
 .../apache/solr/response/CSVResponseWriter.java |   10 +-
 .../apache/solr/response/SchemaXmlWriter.java   |   27 +-
 .../apache/solr/response/SolrQueryResponse.java |    4 +
 .../solr/response/TextResponseWriter.java       |   10 +-
 .../transform/ValueAugmenterFactory.java        |    4 +-
 .../org/apache/solr/rest/ManagedResource.java   |   24 +-
 .../solr/schema/AbstractSpatialFieldType.java   |    7 +-
 .../java/org/apache/solr/schema/BBoxField.java  |    6 +-
 .../org/apache/solr/schema/DateRangeField.java  |   15 +-
 .../java/org/apache/solr/schema/FieldType.java  |    7 +-
 .../org/apache/solr/schema/IndexSchema.java     |  209 ++-
 .../apache/solr/schema/ManagedIndexSchema.java  |    2 +-
 .../solr/schema/ManagedIndexSchemaFactory.java  |    4 +-
 .../org/apache/solr/schema/SchemaManager.java   |   99 +-
 .../schema/SpatialPointVectorFieldType.java     |   23 +-
 .../java/org/apache/solr/schema/StrField.java   |   26 +-
 .../org/apache/solr/schema/TrieDateField.java   |    5 +-
 .../java/org/apache/solr/schema/TrieField.java  |  129 +-
 .../apache/solr/schema/ZkIndexSchemaReader.java |   56 +-
 .../org/apache/solr/search/DocSetCollector.java |   84 +-
 .../apache/solr/search/FunctionRangeQuery.java  |    2 +-
 .../solr/search/GraphTermsQParserPlugin.java    |  292 ++++
 .../org/apache/solr/search/QParserPlugin.java   |    1 +
 .../apache/solr/search/SolrFieldCacheMBean.java |   21 +-
 .../apache/solr/search/SolrReturnFields.java    |   17 +-
 .../apache/solr/search/ValueSourceParser.java   |   37 +-
 .../apache/solr/search/facet/FacetRange.java    |    5 +-
 .../search/function/ValueSourceRangeFilter.java |    8 +-
 .../EndResultTransformer.java                   |    2 +-
 .../solr/security/AuthenticationPlugin.java     |    3 -
 .../solr/security/AuthorizationContext.java     |    3 +
 .../security/AutorizationEditOperation.java     |  176 ++
 .../solr/security/HttpClientBuilderPlugin.java  |   37 +
 .../security/HttpClientInterceptorPlugin.java   |   30 -
 .../apache/solr/security/KerberosPlugin.java    |   17 +-
 .../solr/security/PKIAuthenticationPlugin.java  |   45 +-
 .../org/apache/solr/security/Permission.java    |  119 ++
 .../solr/security/PermissionNameProvider.java   |   74 +
 .../security/RuleBasedAuthorizationPlugin.java  |  294 +---
 .../org/apache/solr/servlet/HttpSolrCall.java   |   30 +-
 .../apache/solr/servlet/LoadAdminUiServlet.java |    9 +-
 .../solr/servlet/ServletInputStreamWrapper.java |  105 ++
 .../servlet/ServletOutputStreamWrapper.java     |  140 ++
 .../apache/solr/servlet/SolrDispatchFilter.java |   92 +
 .../apache/solr/servlet/SolrRequestParsers.java |   12 +-
 .../solr/store/blockcache/BlockDirectory.java   |   32 +-
 .../solr/update/DirectUpdateHandler2.java       |    4 +-
 .../org/apache/solr/update/HdfsUpdateLog.java   |   13 +-
 .../apache/solr/update/SolrCmdDistributor.java  |   45 +-
 .../org/apache/solr/update/SolrCoreState.java   |    2 +-
 .../solr/update/StreamingSolrClients.java       |   66 +-
 .../apache/solr/update/UpdateShardHandler.java  |   57 +-
 .../processor/DistributedUpdateProcessor.java   |  144 +-
 .../DistributedUpdateProcessorFactory.java      |   20 +
 .../DocExpirationUpdateProcessorFactory.java    |   42 +-
 .../FieldMutatingUpdateProcessorFactory.java    |    2 +-
 .../processor/TolerantUpdateProcessor.java      |  415 +++++
 .../TolerantUpdateProcessorFactory.java         |  142 ++
 .../org/apache/solr/util/CommandOperation.java  |   27 +
 .../org/apache/solr/util/DateFormatUtil.java    |  245 ---
 .../org/apache/solr/util/DateMathParser.java    |   88 +-
 .../org/apache/solr/util/SimplePostTool.java    |    6 +-
 .../src/java/org/apache/solr/util/SolrCLI.java  |   52 +-
 .../org/apache/solr/util/SolrLogLayout.java     |   10 +-
 .../src/resources/WellKnownPermissions.json     |   47 -
 solr/core/src/test-files/log4j.properties       |    4 +-
 .../solr/collection1/conf/schema-docValues.xml  |   13 +-
 .../solr/collection1/conf/schema15.xml          |    1 +
 .../solr/collection1/conf/schema_latest.xml     |   51 +-
 ...lrconfig-distrib-update-processor-chains.xml |   85 +
 .../conf/solrconfig-tolerant-update-minimal.xml |   40 +
 .../conf/solrconfig-update-processor-chains.xml |   17 +
 .../cloud-managed/conf/managed-schema           |   31 +
 .../cloud-managed/conf/solrconfig.xml           |   51 +
 .../configset-1/conf/schema-minimal.xml         |   25 -
 .../configset-1/conf/solrconfig-minimal.xml     |   56 -
 .../solr/configsets/resource-sharing/schema.xml |   25 +
 .../configsets/resource-sharing/solrconfig.xml  |   51 +
 .../org/apache/solr/BasicFunctionalityTest.java |   29 +-
 .../org/apache/solr/TestDistributedSearch.java  |   12 +-
 .../org/apache/solr/TestTolerantSearch.java     |    6 +-
 .../core/src/test/org/apache/solr/TestTrie.java |   13 +-
 .../solr/client/solrj/ConnectionReuseTest.java  |  195 +--
 .../solrj/embedded/TestJettySolrRunner.java     |   13 +-
 .../apache/solr/cloud/AliasIntegrationTest.java |   14 +-
 .../test/org/apache/solr/cloud/AssignTest.java  |    2 +-
 .../solr/cloud/AsyncMigrateRouteKeyTest.java    |    2 +-
 .../solr/cloud/BaseCdcrDistributedZkTest.java   |   13 +-
 .../solr/cloud/BasicDistributedZk2Test.java     |    9 +-
 .../solr/cloud/BasicDistributedZkTest.java      |  115 +-
 .../cloud/ChaosMonkeyNothingIsSafeTest.java     |   54 +-
 .../apache/solr/cloud/CollectionReloadTest.java |   85 +-
 .../cloud/CollectionsAPIDistributedZkTest.java  |   55 +-
 .../solr/cloud/CollectionsAPISolrJTest.java     |    4 +-
 ...ConcurrentDeleteAndCreateCollectionTest.java |    6 +-
 .../solr/cloud/ConnectionManagerTest.java       |   40 +-
 .../solr/cloud/CreateCollectionCleanupTest.java |   84 +
 .../apache/solr/cloud/CustomCollectionTest.java |   12 +-
 .../solr/cloud/DeleteInactiveReplicaTest.java   |    2 +-
 .../apache/solr/cloud/DeleteReplicaTest.java    |    4 +-
 .../org/apache/solr/cloud/DeleteShardTest.java  |    6 +-
 ...DistribDocExpirationUpdateProcessorTest.java |    3 +-
 .../solr/cloud/DistributedVersionInfoTest.java  |   24 +-
 .../org/apache/solr/cloud/ForceLeaderTest.java  |    2 +-
 .../cloud/FullSolrCloudDistribCmdsTest.java     |    2 +-
 .../apache/solr/cloud/HttpPartitionTest.java    |   28 +-
 .../cloud/LeaderFailoverAfterPartitionTest.java |    9 +-
 .../LeaderInitiatedRecoveryOnCommitTest.java    |    2 +-
 ...aderInitiatedRecoveryOnShardRestartTest.java |    2 +-
 .../apache/solr/cloud/MigrateRouteKeyTest.java  |    4 +-
 .../solr/cloud/OverseerTaskQueueTest.java       |   67 +
 .../org/apache/solr/cloud/OverseerTest.java     |    3 +-
 .../solr/cloud/ReplicationFactorTest.java       |    2 +-
 .../org/apache/solr/cloud/SSLMigrationTest.java |    4 +-
 .../solr/cloud/SaslZkACLProviderTest.java       |    2 +
 .../cloud/SegmentTerminateEarlyTestState.java   |    6 +-
 .../org/apache/solr/cloud/ShardSplitTest.java   |   34 +-
 .../org/apache/solr/cloud/SyncSliceTest.java    |    2 +-
 .../solr/cloud/TestAuthenticationFramework.java |   52 +-
 .../solr/cloud/TestCloudDeleteByQuery.java      |   28 +-
 .../apache/solr/cloud/TestCloudPivotFacet.java  |   23 +-
 .../apache/solr/cloud/TestConfigSetsAPI.java    |   20 +-
 .../cloud/TestConfigSetsAPIExclusivity.java     |    4 +-
 .../solr/cloud/TestConfigSetsAPIZkFailure.java  |    4 +-
 .../org/apache/solr/cloud/TestCryptoKeys.java   |    9 +-
 .../cloud/TestMiniSolrCloudClusterBase.java     |    2 +-
 .../solr/cloud/TestMiniSolrCloudClusterSSL.java |    2 +-
 .../cloud/TestOnReconnectListenerSupport.java   |  155 ++
 .../cloud/TestRandomRequestDistribution.java    |  104 +-
 .../apache/solr/cloud/TestRebalanceLeaders.java |   18 +-
 .../cloud/TestRequestStatusCollectionAPI.java   |    2 +-
 .../cloud/TestSolrCloudWithKerberosAlt.java     |   13 +-
 .../apache/solr/cloud/TestStressLiveNodes.java  |  255 +++
 .../cloud/TestTolerantUpdateProcessorCloud.java | 1081 ++++++++++++
 .../TestTolerantUpdateProcessorRandomCloud.java |  392 +++++
 .../solr/cloud/UnloadDistributedZkTest.java     |   30 +-
 .../test/org/apache/solr/cloud/ZkCLITest.java   |   43 +-
 .../org/apache/solr/cloud/ZkControllerTest.java |    9 +-
 .../HdfsWriteToMultipleCollectionsTest.java     |    2 +-
 .../apache/solr/cloud/hdfs/StressHdfsTest.java  |   19 +-
 .../solr/cloud/overseer/ZkStateReaderTest.java  |   65 +-
 .../solr/core/BlobRepositoryCloudTest.java      |  118 ++
 .../solr/core/BlobRepositoryMockingTest.java    |  165 ++
 .../solr/core/OpenCloseCoreStressTest.java      |    6 +-
 .../test/org/apache/solr/core/SolrCoreTest.java |   41 +
 .../org/apache/solr/core/TestBadConfig.java     |    5 +
 .../org/apache/solr/core/TestCoreContainer.java |    1 -
 .../apache/solr/core/TestDynamicLoading.java    |   13 +-
 .../solr/core/TestImplicitCoreProperties.java   |    4 +-
 .../apache/solr/core/TestJmxMonitoredMap.java   |   37 +
 .../apache/solr/core/TestSolrConfigHandler.java |    2 +-
 .../apache/solr/handler/CheckBackupStatus.java  |    2 +-
 .../apache/solr/handler/TestConfigReload.java   |    7 +-
 .../solr/handler/TestReplicationHandler.java    |   30 +-
 .../handler/TestReplicationHandlerBackup.java   |    9 +-
 .../apache/solr/handler/TestReqParamsAPI.java   |    7 +-
 .../apache/solr/handler/TestRestoreCore.java    |    7 +-
 .../org/apache/solr/handler/TestSQLHandler.java |   74 +-
 .../handler/TestSolrConfigHandlerCloud.java     |    7 +-
 .../TestSolrConfigHandlerConcurrent.java        |    7 +-
 .../admin/CoreAdminCreateDiscoverTest.java      |    1 +
 .../handler/admin/CoreAdminHandlerTest.java     |    4 +-
 .../handler/admin/SecurityConfHandlerTest.java  |   30 +-
 .../DistributedDebugComponentTest.java          |    6 +-
 .../DistributedFacetPivotLargeTest.java         |    3 +-
 .../DistributedQueryElevationComponentTest.java |    4 +-
 .../component/ResourceSharingTestComponent.java |  149 ++
 .../apache/solr/request/SimpleFacetsTest.java   |    9 +-
 .../solr/response/TestCSVResponseWriter.java    |   14 +-
 .../schema/TestCopyFieldCollectionResource.java |   34 +
 .../TestDynamicFieldCollectionResource.java     |   24 +
 .../schema/TestFieldCollectionResource.java     |   51 +-
 .../solr/rest/schema/TestFieldResource.java     |    6 +
 .../org/apache/solr/schema/DateFieldTest.java   |  148 +-
 .../apache/solr/schema/DocValuesMultiTest.java  |   71 +-
 .../org/apache/solr/schema/DocValuesTest.java   |  437 ++++-
 .../solr/schema/TestBulkSchemaConcurrent.java   |    7 +-
 .../TestCloudManagedSchemaConcurrent.java       |   19 +-
 .../apache/solr/schema/TestCloudSchemaless.java |    7 +-
 .../solr/schema/TestManagedSchemaAPI.java       |  101 ++
 .../solr/schema/TestUseDocValuesAsStored.java   |    9 +-
 .../solr/search/AnalyticsMergeStrategyTest.java |    7 +-
 .../apache/solr/search/QueryEqualityTest.java   |    9 +
 .../apache/solr/search/ReturnFieldsTest.java    |    8 +
 .../test/org/apache/solr/search/TestDocSet.java |   27 +-
 .../search/TestGraphTermsQParserPlugin.java     |  135 ++
 .../org/apache/solr/search/TestRangeQuery.java  |    5 +-
 .../org/apache/solr/search/TestRealTimeGet.java |   49 +-
 .../org/apache/solr/search/TestRecovery.java    |    7 +-
 .../apache/solr/search/TestRecoveryHdfs.java    |   13 +-
 .../solr/search/TestSolrFieldCacheMBean.java    |   81 +
 .../test/org/apache/solr/search/TestSolrJ.java  |    4 +-
 .../solr/search/stats/TestDistribIDF.java       |   82 +-
 .../solr/security/BasicAuthIntegrationTest.java |   45 +-
 .../solr/security/MockAuthorizationPlugin.java  |   10 +-
 .../PKIAuthenticationIntegrationTest.java       |    2 -
 .../security/TestAuthorizationFramework.java    |    3 +-
 .../TestRuleBasedAuthorizationPlugin.java       |  143 +-
 .../apache/solr/update/AddBlockUpdateTest.java  |   20 +-
 .../org/apache/solr/update/AutoCommitTest.java  |    2 +-
 .../test/org/apache/solr/update/TestUpdate.java |   16 +-
 .../update/processor/AtomicUpdatesTest.java     |   48 +-
 .../processor/TestNamedUpdateProcessors.java    |   10 +-
 .../processor/TolerantUpdateProcessorTest.java  |  447 +++++
 .../apache/solr/util/DateMathParserTest.java    |  125 +-
 .../apache/solr/util/TestSolrCLIRunExample.java |   11 +-
 .../org/apache/solr/util/TestTestInjection.java |    2 +-
 solr/example/films/README.txt                   |    2 +
 solr/licenses/junit4-ant-2.3.2.jar.sha1         |    1 -
 solr/licenses/junit4-ant-2.3.4.jar.sha1         |    1 +
 .../randomizedtesting-runner-2.3.2.jar.sha1     |    1 -
 .../randomizedtesting-runner-2.3.4.jar.sha1     |    1 +
 solr/server/etc/jetty-http.xml                  |    2 +-
 solr/server/etc/jetty-https.xml                 |    2 +-
 solr/server/etc/jetty.xml                       |    2 +-
 .../org/apache/solr/client/solrj/SolrQuery.java |   21 +-
 .../solr/client/solrj/impl/CloudSolrClient.java |  311 +++-
 .../solrj/impl/ConcurrentUpdateSolrClient.java  |  229 ++-
 .../client/solrj/impl/HttpClientConfigurer.java |  100 --
 .../solr/client/solrj/impl/HttpClientUtil.java  |  438 ++---
 .../solr/client/solrj/impl/HttpSolrClient.java  |  196 ++-
 .../solrj/impl/Krb5HttpClientBuilder.java       |  178 ++
 .../solrj/impl/Krb5HttpClientConfigurer.java    |  153 --
 .../client/solrj/impl/LBHttpSolrClient.java     |  159 +-
 .../solrj/impl/SolrHttpClientBuilder.java       |   91 +
 .../impl/SolrHttpClientContextBuilder.java      |   96 ++
 .../client/solrj/impl/XMLResponseParser.java    |   27 +-
 .../solr/client/solrj/io/SolrClientCache.java   |    9 +-
 .../client/solrj/io/comp/FieldComparator.java   |   55 +-
 .../solrj/io/comp/MultipleFieldComparator.java  |   12 +
 .../solr/client/solrj/io/eq/FieldEqualitor.java |   17 +-
 .../solrj/io/eq/MultipleFieldEqualitor.java     |   12 +
 .../solrj/io/graph/GatherNodesStream.java       |  653 ++++++++
 .../apache/solr/client/solrj/io/graph/Node.java |   90 +
 .../solrj/io/graph/ShortestPathStream.java      |  515 ++++++
 .../solr/client/solrj/io/graph/Traversal.java   |   96 ++
 .../solrj/io/graph/TraversalIterator.java       |  120 ++
 .../client/solrj/io/graph/package-info.java     |   22 +
 .../client/solrj/io/ops/ConcatOperation.java    |   15 +-
 .../client/solrj/io/ops/DistinctOperation.java  |   33 +-
 .../client/solrj/io/ops/GroupOperation.java     |   42 +-
 .../client/solrj/io/ops/ReplaceOperation.java   |    6 +
 .../solrj/io/ops/ReplaceWithFieldOperation.java |   12 +
 .../solrj/io/ops/ReplaceWithValueOperation.java |   12 +
 .../client/solrj/io/ops/StreamOperation.java    |    3 -
 .../client/solrj/io/sql/ConnectionImpl.java     |   18 +-
 .../solrj/io/sql/DatabaseMetaDataImpl.java      |    6 +-
 .../solrj/io/sql/PreparedStatementImpl.java     |  394 +++++
 .../solr/client/solrj/io/sql/ResultSetImpl.java |   78 +-
 .../solr/client/solrj/io/sql/StatementImpl.java |   65 +-
 .../client/solrj/io/stream/CloudSolrStream.java |   39 +-
 .../solrj/io/stream/ComplementStream.java       |   57 +-
 .../client/solrj/io/stream/DaemonStream.java    |   90 +-
 .../client/solrj/io/stream/ExceptionStream.java |   14 +
 .../client/solrj/io/stream/FacetStream.java     |   42 +-
 .../client/solrj/io/stream/HashJoinStream.java  |   41 +-
 .../client/solrj/io/stream/IntersectStream.java |   56 +-
 .../solr/client/solrj/io/stream/JDBCStream.java |   37 +
 .../solr/client/solrj/io/stream/JoinStream.java |   33 +-
 .../client/solrj/io/stream/MergeStream.java     |   35 +-
 .../client/solrj/io/stream/ParallelStream.java  |   41 +-
 .../client/solrj/io/stream/PushBackStream.java  |    7 +-
 .../client/solrj/io/stream/RandomStream.java    |  237 +++
 .../solr/client/solrj/io/stream/RankStream.java |   43 +-
 .../client/solrj/io/stream/ReducerStream.java   |   37 +-
 .../client/solrj/io/stream/RollupStream.java    |   43 +-
 .../client/solrj/io/stream/SelectStream.java    |   41 +-
 .../solr/client/solrj/io/stream/SolrStream.java |   22 +-
 .../solr/client/solrj/io/stream/SortStream.java |  199 +++
 .../client/solrj/io/stream/StatsStream.java     |   34 +-
 .../client/solrj/io/stream/StreamContext.java   |    4 +
 .../client/solrj/io/stream/TopicStream.java     |   77 +-
 .../client/solrj/io/stream/TupleStream.java     |   10 +
 .../client/solrj/io/stream/UniqueStream.java    |   37 +-
 .../client/solrj/io/stream/UpdateStream.java    |   53 +-
 .../solrj/io/stream/expr/Explanation.java       |  158 ++
 .../solrj/io/stream/expr/Expressible.java       |    9 +
 .../solrj/io/stream/expr/StreamExplanation.java |   72 +
 .../client/solrj/io/stream/metrics/Bucket.java  |   11 +-
 .../solrj/io/stream/metrics/CountMetric.java    |    4 +
 .../solrj/io/stream/metrics/MaxMetric.java      |    5 +
 .../solrj/io/stream/metrics/MeanMetric.java     |    5 +
 .../client/solrj/io/stream/metrics/Metric.java  |   21 +
 .../solrj/io/stream/metrics/MinMetric.java      |    7 +-
 .../solrj/io/stream/metrics/SumMetric.java      |    5 +
 .../solrj/request/CollectionAdminRequest.java   |   61 +
 .../solrj/response/schema/SchemaResponse.java   |   64 +-
 .../solr/client/solrj/util/ClientUtils.java     |   20 +-
 .../solrj/util/SolrIdentifierValidator.java     |   12 +-
 .../solr/common/ToleratedUpdateError.java       |  197 +++
 .../apache/solr/common/cloud/ClusterState.java  |   48 +-
 .../solr/common/cloud/ClusterStateUtil.java     |    1 +
 .../solr/common/cloud/ConnectionManager.java    |    8 +-
 .../apache/solr/common/cloud/DocCollection.java |    6 +
 .../apache/solr/common/cloud/OnReconnect.java   |    9 +-
 .../solr/common/cloud/ZkConfigManager.java      |   27 +-
 .../apache/solr/common/cloud/ZkStateReader.java |  133 +-
 .../solr/common/params/CoreAdminParams.java     |   16 +-
 .../org/apache/solr/common/util/DateUtil.java   |  259 ---
 .../apache/solr/common/util/JavaBinCodec.java   |    8 +-
 .../org/apache/solr/common/util/NamedList.java  |   69 +
 solr/solrj/src/test-files/log4j.properties      |    2 +
 .../solr/collection1/conf/schema-streaming.xml  |    5 +
 .../client/solrj/SolrExampleBinaryTest.java     |    4 +-
 .../solr/client/solrj/SolrExampleTests.java     |    2 +-
 .../solr/client/solrj/SolrExampleXMLTest.java   |    4 +-
 .../solr/client/solrj/SolrExceptionTest.java    |   11 +-
 .../apache/solr/client/solrj/SolrQueryTest.java |   18 +-
 .../client/solrj/SolrSchemalessExampleTest.java |    9 +-
 .../solr/client/solrj/TestLBHttpSolrClient.java |   23 +-
 .../client/solrj/embedded/JettyWebappTest.java  |    7 +-
 .../solrj/embedded/SolrExampleJettyTest.java    |    5 +-
 .../embedded/SolrExampleStreamingTest.java      |   23 +-
 .../solrj/impl/BasicHttpSolrClientTest.java     |  112 +-
 .../solrj/impl/CloudSolrClientBuilderTest.java  |   90 +
 .../client/solrj/impl/CloudSolrClientTest.java  |   29 +-
 .../ConcurrentUpdateSolrClientBuilderTest.java  |   33 +
 .../impl/ConcurrentUpdateSolrClientTest.java    |  115 +-
 .../solrj/impl/ExternalHttpClientTest.java      |   75 -
 .../client/solrj/impl/HttpClientUtilTest.java   |  162 --
 .../solrj/impl/HttpSolrClientBuilderTest.java   |   76 +
 .../solrj/impl/LBHttpSolrClientBuilderTest.java |   65 +
 .../client/solrj/impl/LBHttpSolrClientTest.java |   17 +-
 .../solrj/io/graph/GraphExpressionTest.java     |  821 +++++++++
 .../solr/client/solrj/io/graph/GraphTest.java   |  387 +++++
 .../solr/client/solrj/io/sql/JdbcTest.java      |   87 +-
 .../solrj/io/stream/RecordCountStream.java      |   36 +-
 .../solrj/io/stream/StreamExpressionTest.java   |  114 ++
 .../StreamExpressionToExplanationTest.java      |  290 ++++
 .../client/solrj/io/stream/StreamingTest.java   |  124 +-
 .../solr/client/solrj/request/SchemaTest.java   |    9 +-
 .../solrj/response/QueryResponseTest.java       |    7 +-
 .../solr/common/TestToleratedUpdateError.java   |  193 +++
 .../solr/common/cloud/TestZkConfigManager.java  |    7 +
 .../apache/solr/common/util/TestDateUtil.java   |   35 -
 .../solr/BaseDistributedSearchTestCase.java     |   63 +-
 .../java/org/apache/solr/SolrJettyTestBase.java |   20 +-
 .../java/org/apache/solr/SolrTestCaseHS.java    |   40 +-
 .../java/org/apache/solr/SolrTestCaseJ4.java    |  166 +-
 .../cloud/AbstractFullDistribZkTestBase.java    |   97 +-
 .../apache/solr/cloud/MiniSolrCloudCluster.java |   19 +-
 .../apache/solr/cloud/SolrCloudTestCase.java    |    4 +-
 .../apache/solr/util/RESTfulServerProvider.java |    2 +-
 .../java/org/apache/solr/util/RestTestBase.java |    7 +-
 .../org/apache/solr/util/RestTestHarness.java   |    4 +-
 .../org/apache/solr/util/SSLTestConfig.java     |   79 +-
 solr/webapp/web/css/angular/menu.css            |    1 +
 solr/webapp/web/css/angular/stream.css          |  233 +++
 solr/webapp/web/index.html                      |    7 +-
 solr/webapp/web/js/angular/app.js               |    8 +
 solr/webapp/web/js/angular/controllers/query.js |    9 +-
 .../webapp/web/js/angular/controllers/stream.js |  240 +++
 solr/webapp/web/js/angular/services.js          |    4 +-
 solr/webapp/web/partials/schema.html            |    2 +-
 solr/webapp/web/partials/stream.html            |   64 +
 780 files changed, 36552 insertions(+), 13728 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e54718df/solr/core/src/java/org/apache/solr/cloud/Overseer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e54718df/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e54718df/solr/core/src/java/org/apache/solr/cloud/overseer/ClusterStateMutator.java
----------------------------------------------------------------------

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

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e54718df/solr/core/src/test/org/apache/solr/handler/TestRestoreCore.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e54718df/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
----------------------------------------------------------------------

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

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e54718df/solr/solrj/src/java/org/apache/solr/common/params/CoreAdminParams.java
----------------------------------------------------------------------
diff --cc solr/solrj/src/java/org/apache/solr/common/params/CoreAdminParams.java
index 4219277,f1728db..716dfee
--- a/solr/solrj/src/java/org/apache/solr/common/params/CoreAdminParams.java
+++ b/solr/solrj/src/java/org/apache/solr/common/params/CoreAdminParams.java
@@@ -128,11 -128,18 +128,21 @@@ public abstract class CoreAdminParam
      REJOINLEADERELECTION,
      //internal API used by force shard leader election
      FORCEPREPAREFORLEADERSHIP,
 -    INVOKE;
 +    INVOKE,
 +    //Internal APIs to backup and restore a core
 +    BACKUPCORE,
 +    RESTORECORE;
  
+     public final boolean isRead;
+ 
+     CoreAdminAction(boolean isRead) {
+       this.isRead = isRead;
+     }
+ 
+     CoreAdminAction() {
+       this.isRead = false;
+     }
+ 
      public static CoreAdminAction get( String p ) {
        if (p != null) {
          try {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e54718df/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
----------------------------------------------------------------------


[10/50] [abbrv] lucene-solr:solr-5750: SOLR-8933: Solr should not close container streams.

Posted by ds...@apache.org.
SOLR-8933: Solr should not close container streams.


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

Branch: refs/heads/solr-5750
Commit: f3de22377486e88ed12427c3bbd3a89c7c051328
Parents: 545a140
Author: markrmiller <ma...@apache.org>
Authored: Thu Apr 21 08:34:14 2016 -0400
Committer: markrmiller <ma...@apache.org>
Committed: Thu Apr 21 11:53:58 2016 -0400

----------------------------------------------------------------------
 .../replicator/http/ReplicationService.java     |   1 +
 solr/CHANGES.txt                                |   2 +
 .../org/apache/solr/servlet/HttpSolrCall.java   |  11 +-
 .../apache/solr/servlet/LoadAdminUiServlet.java |   9 +-
 .../solr/servlet/ServletInputStreamWrapper.java | 105 ++++++++++++++
 .../servlet/ServletOutputStreamWrapper.java     | 140 +++++++++++++++++++
 .../apache/solr/servlet/SolrDispatchFilter.java |  92 ++++++++++++
 .../apache/solr/servlet/SolrRequestParsers.java |  12 +-
 .../client/solrj/embedded/JettyWebappTest.java  |   4 +-
 9 files changed, 367 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f3de2237/lucene/replicator/src/java/org/apache/lucene/replicator/http/ReplicationService.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/http/ReplicationService.java b/lucene/replicator/src/java/org/apache/lucene/replicator/http/ReplicationService.java
index e392445..39cd994 100644
--- a/lucene/replicator/src/java/org/apache/lucene/replicator/http/ReplicationService.java
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/http/ReplicationService.java
@@ -148,6 +148,7 @@ public class ReplicationService {
       throw new ServletException("unrecognized shard ID " + pathElements[SHARD_IDX]);
     }
     
+    // SOLR-8933 Don't close this stream.
     ServletOutputStream resOut = resp.getOutputStream();
     try {
       switch (action) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f3de2237/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 9c8d03b..6034851 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -203,6 +203,8 @@ Other Changes
 
 * SOLR-8929: Add an idea module for solr/server to enable launching start.jar (Scott Blum, Steve Rowe)
 
+* SOLR-8933: Solr should not close container streams. (Mike Drob, Uwe Schindler, Mark Miller)
+
 ==================  6.0.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/f3de2237/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
index 4687154..eb90762 100644
--- a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
+++ b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
@@ -17,9 +17,9 @@
 package org.apache.solr.servlet;
 
 import javax.servlet.ServletInputStream;
-
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
+
 import java.io.EOFException;
 import java.io.IOException;
 import java.io.InputStream;
@@ -41,7 +41,10 @@ import java.util.Random;
 import java.util.Set;
 
 import com.google.common.collect.ImmutableSet;
+
 import org.apache.commons.io.IOUtils;
+import org.apache.commons.io.input.CloseShieldInputStream;
+import org.apache.commons.io.output.CloseShieldOutputStream;
 import org.apache.commons.lang.StringUtils;
 import org.apache.http.Header;
 import org.apache.http.HeaderIterator;
@@ -534,7 +537,8 @@ public class HttpSolrCall {
       } else if (isPostOrPutRequest) {
         HttpEntityEnclosingRequestBase entityRequest =
             "POST".equals(req.getMethod()) ? new HttpPost(urlstr) : new HttpPut(urlstr);
-        HttpEntity entity = new InputStreamEntity(req.getInputStream(), req.getContentLength());
+        InputStream in = new CloseShieldInputStream(req.getInputStream()); // Prevent close of container streams
+        HttpEntity entity = new InputStreamEntity(in, req.getContentLength());
         entityRequest.setEntity(entity);
         method = entityRequest;
       } else if ("DELETE".equals(req.getMethod())) {
@@ -723,7 +727,8 @@ public class HttpSolrCall {
       }
 
       if (Method.HEAD != reqMethod) {
-        QueryResponseWriterUtil.writeQueryResponse(response.getOutputStream(), responseWriter, solrReq, solrRsp, ct);
+        OutputStream out = new CloseShieldOutputStream(response.getOutputStream()); // Prevent close of container streams, see SOLR-8933
+        QueryResponseWriterUtil.writeQueryResponse(out, responseWriter, solrReq, solrRsp, ct);
       }
       //else http HEAD request, nothing to write out, waited this long just to get ContentType
     } catch (EOFException e) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f3de2237/solr/core/src/java/org/apache/solr/servlet/LoadAdminUiServlet.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/LoadAdminUiServlet.java b/solr/core/src/java/org/apache/solr/servlet/LoadAdminUiServlet.java
index 992dfe2..c496ce1 100644
--- a/solr/core/src/java/org/apache/solr/servlet/LoadAdminUiServlet.java
+++ b/solr/core/src/java/org/apache/solr/servlet/LoadAdminUiServlet.java
@@ -17,6 +17,7 @@
 package org.apache.solr.servlet;
 
 import org.apache.commons.io.IOUtils;
+import org.apache.commons.io.output.CloseShieldOutputStream;
 import org.apache.commons.lang.StringEscapeUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.solr.common.params.CommonParams;
@@ -25,6 +26,7 @@ import org.apache.solr.core.SolrCore;
 
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
+
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStreamWriter;
@@ -49,11 +51,14 @@ public final class LoadAdminUiServlet extends BaseSolrServlet {
     String admin = request.getRequestURI().substring(request.getContextPath().length());
     CoreContainer cores = (CoreContainer) request.getAttribute("org.apache.solr.CoreContainer");
     InputStream in = getServletContext().getResourceAsStream(admin);
+    Writer out = null;
     if(in != null && cores != null) {
       try {
         response.setCharacterEncoding("UTF-8");
         response.setContentType("text/html");
-        Writer out = new OutputStreamWriter(response.getOutputStream(), StandardCharsets.UTF_8);
+
+        // Protect container owned streams from being closed by us, see SOLR-8933
+        out = new OutputStreamWriter(new CloseShieldOutputStream(response.getOutputStream()), StandardCharsets.UTF_8);
 
         String html = IOUtils.toString(in, "UTF-8");
         Package pack = SolrCore.class.getPackage();
@@ -70,9 +75,9 @@ public final class LoadAdminUiServlet extends BaseSolrServlet {
         };
         
         out.write( StringUtils.replaceEach(html, search, replace) );
-        out.flush();
       } finally {
         IOUtils.closeQuietly(in);
+        IOUtils.closeQuietly(out);
       }
     } else {
       response.sendError(404);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f3de2237/solr/core/src/java/org/apache/solr/servlet/ServletInputStreamWrapper.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/ServletInputStreamWrapper.java b/solr/core/src/java/org/apache/solr/servlet/ServletInputStreamWrapper.java
new file mode 100644
index 0000000..d229bf7
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/servlet/ServletInputStreamWrapper.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.solr.servlet;
+
+import java.io.IOException;
+
+import javax.servlet.ReadListener;
+import javax.servlet.ServletInputStream;
+
+import org.apache.solr.common.util.SuppressForbidden;
+
+/**
+ * Provides a convenient extension of the {@link ServletInputStream} class that can be subclassed by developers wishing
+ * to adapt the behavior of a Stream. One such example may be to override {@link #close()} to instead be a no-op as in
+ * SOLR-8933.
+ *
+ * This class implements the Wrapper or Decorator pattern. Methods default to calling through to the wrapped stream.
+ */
+@SuppressForbidden(reason = "delegate methods")
+public class ServletInputStreamWrapper extends ServletInputStream {
+  final ServletInputStream stream;
+  
+  public ServletInputStreamWrapper(ServletInputStream stream) throws IOException {
+    this.stream = stream;
+  }
+  
+  public int hashCode() {
+    return stream.hashCode();
+  }
+
+  public boolean equals(Object obj) {
+    return stream.equals(obj);
+  }
+
+  public int available() throws IOException {
+    return stream.available();
+  }
+
+  public void close() throws IOException {
+    stream.close();
+  }
+
+  public boolean isFinished() {
+    return stream.isFinished();
+  }
+
+  public boolean isReady() {
+    return stream.isReady();
+  }
+
+  public int read() throws IOException {
+    return stream.read();
+  }
+
+  public int read(byte[] b) throws IOException {
+    return stream.read(b);
+  }
+
+  public int read(byte[] b, int off, int len) throws IOException {
+    return stream.read(b, off, len);
+  }
+
+  public void mark(int readlimit) {
+    stream.mark(readlimit);
+  }
+
+  public boolean markSupported() {
+    return stream.markSupported();
+  }
+
+  public int readLine(byte[] b, int off, int len) throws IOException {
+    return stream.readLine(b, off, len);
+  }
+
+  public void reset() throws IOException {
+    stream.reset();
+  }
+
+  public void setReadListener(ReadListener arg0) {
+    stream.setReadListener(arg0);
+  }
+
+  public long skip(long n) throws IOException {
+    return stream.skip(n);
+  }
+
+  public String toString() {
+    return stream.toString();
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f3de2237/solr/core/src/java/org/apache/solr/servlet/ServletOutputStreamWrapper.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/ServletOutputStreamWrapper.java b/solr/core/src/java/org/apache/solr/servlet/ServletOutputStreamWrapper.java
new file mode 100644
index 0000000..d12c3bd
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/servlet/ServletOutputStreamWrapper.java
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.servlet;
+
+import java.io.IOException;
+
+import javax.servlet.ServletOutputStream;
+import javax.servlet.WriteListener;
+
+import org.apache.solr.common.util.SuppressForbidden;
+
+/**
+ * Provides a convenient extension of the {@link ServletOutputStream} class that can be subclassed by developers wishing
+ * to adapt the behavior of a Stream. One such example may be to override {@link #close()} to instead be a no-op as in
+ * SOLR-8933.
+ *
+ * This class implements the Wrapper or Decorator pattern. Methods default to calling through to the wrapped stream.
+ */
+@SuppressForbidden(reason = "delegate methods")
+public class ServletOutputStreamWrapper extends ServletOutputStream {
+  final ServletOutputStream stream;
+  
+  public ServletOutputStreamWrapper(ServletOutputStream stream) {
+    this.stream = stream;
+  }
+
+  public int hashCode() {
+    return stream.hashCode();
+  }
+
+  public boolean equals(Object obj) {
+    return stream.equals(obj);
+  }
+
+  public void flush() throws IOException {
+    stream.flush();
+  }
+
+  public void close() throws IOException {
+    stream.close();
+  }
+
+  public boolean isReady() {
+    return stream.isReady();
+  }
+
+  public void print(boolean arg0) throws IOException {
+    stream.print(arg0);
+  }
+
+  public void print(char c) throws IOException {
+    stream.print(c);
+  }
+
+  public void print(double d) throws IOException {
+    stream.print(d);
+  }
+
+  public void print(float f) throws IOException {
+    stream.print(f);
+  }
+
+  public void print(int i) throws IOException {
+    stream.print(i);
+  }
+
+  public void print(long l) throws IOException {
+    stream.print(l);
+  }
+
+  public void print(String arg0) throws IOException {
+    stream.print(arg0);
+  }
+
+  public void println() throws IOException {
+    stream.println();
+  }
+
+  public void println(boolean b) throws IOException {
+    stream.println(b);
+  }
+
+  public void println(char c) throws IOException {
+    stream.println(c);
+  }
+
+  public void println(double d) throws IOException {
+    stream.println(d);
+  }
+
+  public void println(float f) throws IOException {
+    stream.println(f);
+  }
+
+  public void println(int i) throws IOException {
+    stream.println(i);
+  }
+
+  public void println(long l) throws IOException {
+    stream.println(l);
+  }
+
+  public void println(String s) throws IOException {
+    stream.println(s);
+  }
+
+  public void setWriteListener(WriteListener arg0) {
+    stream.setWriteListener(arg0);
+  }
+
+  public void write(int b) throws IOException {
+    stream.write(b);
+  }
+
+  public void write(byte[] b) throws IOException {
+    stream.write(b);
+  }
+
+  public void write(byte[] b, int off, int len) throws IOException {
+    stream.write(b, off, len);
+  }
+
+  public String toString() {
+    return stream.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f3de2237/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 7a0e4ef..2d08935 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
@@ -20,12 +20,18 @@ import javax.servlet.FilterChain;
 import javax.servlet.FilterConfig;
 import javax.servlet.ServletException;
 import javax.servlet.ServletInputStream;
+import javax.servlet.ServletOutputStream;
 import javax.servlet.ServletRequest;
 import javax.servlet.ServletResponse;
 import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletRequestWrapper;
 import javax.servlet.http.HttpServletResponse;
+import javax.servlet.http.HttpServletResponseWrapper;
+
 import java.io.ByteArrayInputStream;
 import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.lang.invoke.MethodHandles;
 import java.nio.file.Path;
 import java.nio.file.Paths;
@@ -36,6 +42,8 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import org.apache.commons.io.input.CloseShieldInputStream;
+import org.apache.commons.io.output.CloseShieldOutputStream;
 import org.apache.commons.lang.StringUtils;
 import org.apache.http.client.HttpClient;
 import org.apache.solr.common.SolrException;
@@ -66,6 +74,9 @@ public class SolrDispatchFilter extends BaseSolrFilter {
   protected String abortErrorMessage = null;
   protected HttpClient httpClient;
   private ArrayList<Pattern> excludePatterns;
+  
+  // Effectively immutable
+  private Boolean testMode = null;
 
   /**
    * Enum to define action that needs to be processed.
@@ -80,6 +91,19 @@ public class SolrDispatchFilter extends BaseSolrFilter {
   }
   
   public SolrDispatchFilter() {
+    // turn on test mode when running tests
+    assert testMode = true;
+    
+    if (testMode == null) {
+      testMode = false;
+    } else {
+      String tm = System.getProperty("solr.tests.doContainerStreamCloseAssert");
+      if (tm != null) {
+        testMode = Boolean.parseBoolean(tm);
+      } else {
+        testMode = true;
+      }
+    }
   }
 
   public static final String PROPERTIES_ATTRIBUTE = "solr.properties";
@@ -202,6 +226,10 @@ public class SolrDispatchFilter extends BaseSolrFilter {
       if (wrappedRequest.get() != null) {
         request = wrappedRequest.get();
       }
+
+      request = closeShield(request, retry);
+      response = closeShield(response, retry);
+      
       if (cores.getAuthenticationPlugin() != null) {
         log.debug("User principal: {}", ((HttpServletRequest) request).getUserPrincipal());
       }
@@ -298,4 +326,68 @@ public class SolrDispatchFilter extends BaseSolrFilter {
     }
     return true;
   }
+  
+  /**
+   * Wrap the request's input stream with a close shield, as if by a {@link CloseShieldInputStream}. If this is a
+   * retry, we will assume that the stream has already been wrapped and do nothing.
+   *
+   * @param request The request to wrap.
+   * @param retry If this is an original request or a retry.
+   * @return A request object with an {@link InputStream} that will ignore calls to close.
+   */
+  private ServletRequest closeShield(ServletRequest request, boolean retry) {
+    if (testMode && !retry) {
+      return new HttpServletRequestWrapper((HttpServletRequest) request) {
+        ServletInputStream stream;
+        
+        @Override
+        public ServletInputStream getInputStream() throws IOException {
+          // Lazy stream creation
+          if (stream == null) {
+            stream = new ServletInputStreamWrapper(super.getInputStream()) {
+              @Override
+              public void close() {
+                assert false : "Attempted close of request input stream.";
+              }
+            };
+          }
+          return stream;
+        }
+      };
+    } else {
+      return request;
+    }
+  }
+  
+  /**
+   * Wrap the response's output stream with a close shield, as if by a {@link CloseShieldOutputStream}. If this is a
+   * retry, we will assume that the stream has already been wrapped and do nothing.
+   *
+   * @param response The response to wrap.
+   * @param retry If this response corresponds to an original request or a retry.
+   * @return A response object with an {@link OutputStream} that will ignore calls to close.
+   */
+  private ServletResponse closeShield(ServletResponse response, boolean retry) {
+    if (testMode && !retry) {
+      return new HttpServletResponseWrapper((HttpServletResponse) response) {
+        ServletOutputStream stream;
+        
+        @Override
+        public ServletOutputStream getOutputStream() throws IOException {
+          // Lazy stream creation
+          if (stream == null) {
+            stream = new ServletOutputStreamWrapper(super.getOutputStream()) {
+              @Override
+              public void close() {
+                assert false : "Attempted close of response output stream.";
+              }
+            };
+          }
+          return stream;
+        }
+      };
+    } else {
+      return response;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f3de2237/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java b/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java
index 79c151b..a91f6a2 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java
@@ -17,6 +17,7 @@
 package org.apache.solr.servlet;
 
 import javax.servlet.http.HttpServletRequest;
+
 import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.IOException;
@@ -41,6 +42,7 @@ import java.util.Map;
 import org.apache.commons.fileupload.FileItem;
 import org.apache.commons.fileupload.disk.DiskFileItemFactory;
 import org.apache.commons.fileupload.servlet.ServletFileUpload;
+import org.apache.commons.io.input.CloseShieldInputStream;
 import org.apache.lucene.util.IOUtils;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
@@ -484,7 +486,8 @@ public class SolrRequestParsers
 
     @Override
     public InputStream getStream() throws IOException {
-      return req.getInputStream();
+      // Protect container owned streams from being closed by us, see SOLR-8933
+      return new CloseShieldInputStream(req.getInputStream());
     }
   }
 
@@ -618,7 +621,8 @@ public class SolrRequestParsers
       final Charset charset = (cs == null) ? StandardCharsets.UTF_8 : Charset.forName(cs);
 
       try {
-        in = FastInputStream.wrap( in == null ? req.getInputStream() : in);
+        // Protect container owned streams from being closed by us, see SOLR-8933
+        in = FastInputStream.wrap( in == null ? new CloseShieldInputStream(req.getInputStream()) : in );
 
         final long bytesRead = parseFormDataContent(in, maxLength, charset, map, false);
         if (bytesRead == 0L && totalLength > 0L) {
@@ -737,7 +741,9 @@ public class SolrRequestParsers
       if (formdata.isFormData(req)) {
         String userAgent = req.getHeader("User-Agent");
         boolean isCurl = userAgent != null && userAgent.startsWith("curl/");
-        FastInputStream input = FastInputStream.wrap( req.getInputStream() );
+
+        // Protect container owned streams from being closed by us, see SOLR-8933
+        FastInputStream input = FastInputStream.wrap( new CloseShieldInputStream(req.getInputStream()) );
 
         if (isCurl) {
           SolrParams params = autodetect(req, streams, input);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f3de2237/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/JettyWebappTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/JettyWebappTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/JettyWebappTest.java
index 467b570..6c13e40 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/JettyWebappTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/JettyWebappTest.java
@@ -64,7 +64,8 @@ public class JettyWebappTest extends SolrTestCaseJ4
     super.setUp();
     System.setProperty("solr.solr.home", SolrJettyTestBase.legacyExampleCollection1SolrHome());
     System.setProperty("tests.shardhandler.randomSeed", Long.toString(random().nextLong()));
-
+    System.setProperty("solr.tests.doContainerStreamCloseAssert", "false");
+    
     File dataDir = createTempDir().toFile();
     dataDir.mkdirs();
 
@@ -95,6 +96,7 @@ public class JettyWebappTest extends SolrTestCaseJ4
     } catch( Exception ex ) {}
     System.clearProperty("tests.shardhandler.randomSeed");
     System.clearProperty("solr.data.dir");
+    System.clearProperty("solr.tests.doContainerStreamCloseAssert");
     super.tearDown();
   }
   


[04/50] [abbrv] lucene-solr:solr-5750: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/lucene-solr

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


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

Branch: refs/heads/solr-5750
Commit: 2ff8bb0c8b65148b5d4043a4e795412c7571e460
Parents: 42b4f04 a9a842f
Author: Karl Wright <Da...@gmail.com>
Authored: Thu Apr 21 08:31:14 2016 -0400
Committer: Karl Wright <Da...@gmail.com>
Committed: Thu Apr 21 08:31:14 2016 -0400

----------------------------------------------------------------------
 .gitignore                                      |   2 +
 dev-tools/idea/.idea/modules.xml                |   1 +
 dev-tools/idea/.idea/workspace.xml              |  13 +-
 dev-tools/idea/solr/server/server.iml           |  19 +
 lucene/CHANGES.txt                              |   8 +
 .../lucene/queries/function/FunctionQuery.java  |  19 +-
 .../queries/TestCustomScoreExplanations.java    |  46 +++
 .../lucene/document/InetAddressPoint.java       |  59 ++-
 .../lucene/document/TestInetAddressPoint.java   |  54 +++
 solr/CHANGES.txt                                |  13 +
 .../org/apache/solr/cloud/ZkController.java     |  17 +-
 .../org/apache/solr/handler/StreamHandler.java  |   9 +-
 .../apache/solr/cloud/hdfs/StressHdfsTest.java  |  17 +-
 .../solr/cloud/overseer/ZkStateReaderTest.java  |  60 +++
 .../client/solrj/io/sql/ConnectionImpl.java     |   6 +-
 .../solrj/io/sql/PreparedStatementImpl.java     | 394 +++++++++++++++++++
 .../solr/client/solrj/io/sql/ResultSetImpl.java |  76 +++-
 .../solr/client/solrj/io/sql/StatementImpl.java |  56 ++-
 .../apache/solr/common/cloud/ZkStateReader.java |  36 +-
 .../solr/client/solrj/io/sql/JdbcTest.java      |  67 +++-
 solr/webapp/web/css/angular/stream.css          |  14 +-
 .../webapp/web/js/angular/controllers/stream.js |  23 +-
 solr/webapp/web/partials/stream.html            |   2 +-
 23 files changed, 895 insertions(+), 116 deletions(-)
----------------------------------------------------------------------



[36/50] [abbrv] lucene-solr:solr-5750: SOLR-9016: Fix SolrIdentifierValidator to not accept empty identifiers

Posted by ds...@apache.org.
SOLR-9016: Fix SolrIdentifierValidator to not accept empty identifiers


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

Branch: refs/heads/solr-5750
Commit: 9166647918d26fd75ae48d993e9191ad8d943fe3
Parents: 9d8fad7
Author: Shai Erera <sh...@apache.org>
Authored: Tue Apr 26 17:28:04 2016 +0300
Committer: Shai Erera <sh...@apache.org>
Committed: Tue Apr 26 18:20:47 2016 +0300

----------------------------------------------------------------------
 solr/CHANGES.txt                                        |  2 ++
 .../solr/client/solrj/util/SolrIdentifierValidator.java | 12 +++++-------
 2 files changed, 7 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/91666479/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 101cdfd..4baea2c 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -154,6 +154,8 @@ Bug Fixes
 
 * SOLR-9029: fix rare ZkStateReader visibility race during collection state format update (Scott Blum, hossman)
 
+* SOLR-9016: Fix SolrIdentifierValidator to not allow empty identifiers. (Shai Erera)
+
 Optimizations
 ----------------------
 * SOLR-8722: Don't force a full ZkStateReader refresh on every Overseer operation.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/91666479/solr/solrj/src/java/org/apache/solr/client/solrj/util/SolrIdentifierValidator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/util/SolrIdentifierValidator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/util/SolrIdentifierValidator.java
index 9473a28..d23b8bb 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/util/SolrIdentifierValidator.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/util/SolrIdentifierValidator.java
@@ -28,7 +28,7 @@ import org.apache.solr.common.SolrException;
  * Identifiers are allowed to contain underscores, periods, hyphens, and alphanumeric characters.
  */
 public class SolrIdentifierValidator {
-  final static Pattern identifierPattern = Pattern.compile("^(?!\\-)[\\._A-Za-z0-9\\-]*$");
+  final static Pattern identifierPattern = Pattern.compile("^(?!\\-)[\\._A-Za-z0-9\\-]+$");
 
   public enum IdentifierType {
     SHARD, COLLECTION, CORE, ALIAS
@@ -64,11 +64,9 @@ public class SolrIdentifierValidator {
   }
 
   public static String getIdentifierMessage(IdentifierType identifierType, String name) {
-      return "Invalid " + identifierType.toString().toLowerCase(Locale.ROOT) + ": " + name + ". "
-          + identifierType.toString().toLowerCase(Locale.ROOT)
-          + " names must consist entirely of periods, underscores, hyphens, and alphanumerics";
-
+      String typeStr = identifierType.toString().toLowerCase(Locale.ROOT);
+    return "Invalid " + typeStr + ": [" + name + "]. " + typeStr + " names must consist entirely of periods, "
+        + "underscores, hyphens, and alphanumerics as well not start with a hyphen";
   }
-}
-
 
+}


[28/50] [abbrv] lucene-solr:solr-5750: LUCENE-7251: remove LatLonGrid, remove slow polygon methods, speed up multiple components

Posted by ds...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/837264a4/lucene/sandbox/src/java/org/apache/lucene/document/LatLonTree.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonTree.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonTree.java
deleted file mode 100644
index f7a2927..0000000
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonTree.java
+++ /dev/null
@@ -1,396 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.document;
-
-import java.util.Arrays;
-
-import org.apache.lucene.geo.Polygon;
-import org.apache.lucene.index.PointValues.Relation;
-
-/**
- * 2D polygon implementation represented as a balanced interval tree of edges.
- * <p>
- * contains() and crosses() are still O(n), but for most practical polygons 
- * are much faster than brute force.
- * <p>
- * Loosely based on the algorithm described in <a href="http://www-ma2.upc.es/geoc/Schirra-pointPolygon.pdf">
- * http://www-ma2.upc.es/geoc/Schirra-pointPolygon.pdf</a>.
- */
-// Both Polygon.contains() and Polygon.crossesSlowly() loop all edges, and first check that the edge is within a range.
-// we just organize the edges to do the same computations on the same subset of edges more efficiently. 
-// TODO: clean this up, call it Polygon2D, and remove all the 2D methods from Polygon?
-final class LatLonTree {
-  private final LatLonTree[] holes;
-
-  /** minimum latitude of this polygon's bounding box area */
-  final double minLat;
-  /** maximum latitude of this polygon's bounding box area */
-  final double maxLat;
-  /** minimum longitude of this polygon's bounding box area */
-  final double minLon;
-  /** maximum longitude of this polygon's bounding box area */
-  final double maxLon;
-  
-  /** root node of our tree */
-  final Edge tree;
-
-  // TODO: "pack" all the gons and holes into one tree with separator.
-  // the algorithms support this, but we have to be careful.
-  LatLonTree(Polygon polygon, LatLonTree... holes) {
-    this.holes = holes.clone();
-    this.minLat = polygon.minLat;
-    this.maxLat = polygon.maxLat;
-    this.minLon = polygon.minLon;
-    this.maxLon = polygon.maxLon;
-    
-    // create interval tree of edges
-    this.tree = createTree(polygon.getPolyLats(), polygon.getPolyLons());
-  }
-
-  /** 
-   * Returns true if the point is contained within this polygon.
-   * <p>
-   * See <a href="https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html">
-   * https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html</a> for more information.
-   */
-  boolean contains(double latitude, double longitude) {
-    // check bounding box
-    if (latitude < minLat || latitude > maxLat || longitude < minLon || longitude > maxLon) {
-      return false;
-    }
-    
-    if (tree.contains(latitude, longitude)) {
-      for (LatLonTree hole : holes) {
-        if (hole.contains(latitude, longitude)) {
-          return false;
-        }
-      }
-      return true;
-    }
-    
-    return false;
-  }
-  
-  /** Returns relation to the provided rectangle */
-  Relation relate(double minLat, double maxLat, double minLon, double maxLon) {
-    // if the bounding boxes are disjoint then the shape does not cross
-    if (maxLon < this.minLon || minLon > this.maxLon || maxLat < this.minLat || minLat > this.maxLat) {
-      return Relation.CELL_OUTSIDE_QUERY;
-    }
-    // if the rectangle fully encloses us, we cross.
-    if (minLat <= this.minLat && maxLat >= this.maxLat && minLon <= this.minLon && maxLon >= this.maxLon) {
-      return Relation.CELL_CROSSES_QUERY;
-    }
-    // check any holes
-    for (LatLonTree hole : holes) {
-      Relation holeRelation = hole.relate(minLat, maxLat, minLon, maxLon);
-      if (holeRelation == Relation.CELL_CROSSES_QUERY) {
-        return Relation.CELL_CROSSES_QUERY;
-      } else if (holeRelation == Relation.CELL_INSIDE_QUERY) {
-        return Relation.CELL_OUTSIDE_QUERY;
-      }
-    }
-    // check each corner: if < 4 are present, its cheaper than crossesSlowly
-    int numCorners = numberOfCorners(minLat, maxLat, minLon, maxLon);
-    if (numCorners == 4) {
-      if (tree.crosses(minLat, maxLat, minLon, maxLon)) {
-        return Relation.CELL_CROSSES_QUERY;
-      }
-      return Relation.CELL_INSIDE_QUERY;
-    } else if (numCorners > 0) {
-      return Relation.CELL_CROSSES_QUERY;
-    }
-    
-    // we cross
-    if (tree.crosses(minLat, maxLat, minLon, maxLon)) {
-      return Relation.CELL_CROSSES_QUERY;
-    }
-    
-    return Relation.CELL_OUTSIDE_QUERY;
-  }
-  
-  // returns 0, 4, or something in between
-  private int numberOfCorners(double minLat, double maxLat, double minLon, double maxLon) {
-    int containsCount = 0;
-    if (contains(minLat, minLon)) {
-      containsCount++;
-    }
-    if (contains(minLat, maxLon)) {
-      containsCount++;
-    }
-    if (containsCount == 1) {
-      return containsCount;
-    }
-    if (contains(maxLat, maxLon)) {
-      containsCount++;
-    }
-    if (containsCount == 2) {
-      return containsCount;
-    }
-    if (contains(maxLat, minLon)) {
-      containsCount++;
-    }
-    return containsCount;
-  }
-
-  /** Helper for multipolygon logic: returns true if any of the supplied polygons contain the point */
-  static boolean contains(LatLonTree[] polygons, double latitude, double longitude) {
-    for (LatLonTree polygon : polygons) {
-      if (polygon.contains(latitude, longitude)) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  /** Returns the multipolygon relation for the rectangle */
-  static Relation relate(LatLonTree[] polygons, double minLat, double maxLat, double minLon, double maxLon) {
-    for (LatLonTree polygon : polygons) {
-      Relation relation = polygon.relate(minLat, maxLat, minLon, maxLon);
-      if (relation != Relation.CELL_OUTSIDE_QUERY) {
-        // note: we optimize for non-overlapping multipolygons. so if we cross one,
-        // we won't keep iterating to try to find a contains.
-        return relation;
-      }
-    }
-    return Relation.CELL_OUTSIDE_QUERY;
-  }
-  
-  /** Builds a tree from multipolygon */
-  static LatLonTree[] build(Polygon... polygons) {
-    // TODO: use one tree with separators (carefully!)
-    LatLonTree trees[] = new LatLonTree[polygons.length];
-    for (int i = 0; i < trees.length; i++) {
-      Polygon gon = polygons[i];
-      Polygon gonHoles[] = gon.getHoles();
-      LatLonTree holes[] = new LatLonTree[gonHoles.length];
-      for (int j = 0; j < holes.length; j++) {
-        holes[j] = new LatLonTree(gonHoles[j]);
-      }
-      trees[i] = new LatLonTree(gon, holes);
-    }
-    return trees;
-  }
-  
-  /** 
-   * Internal tree node: represents polygon edge from lat1,lon1 to lat2,lon2.
-   * The sort value is {@code low}, which is the minimum latitude of the edge.
-   * {@code max} stores the maximum latitude of this edge or any children.
-   */
-  static final class Edge {
-    // lat-lon pair (in original order) of the two vertices
-    final double lat1, lat2;
-    final double lon1, lon2;
-    /** min of this edge */
-    final double low;
-    /** max latitude of this edge or any children */
-    double max;
-    
-    /** left child edge, or null */
-    Edge left;
-    /** right child edge, or null */
-    Edge right;
-
-    Edge(double lat1, double lon1, double lat2, double lon2, double low, double max) {
-      this.lat1 = lat1;
-      this.lon1 = lon1;
-      this.lat2 = lat2;
-      this.lon2 = lon2;
-      this.low = low;
-      this.max = max;
-    }
-    
-    /** 
-     * Returns true if the point crosses this edge subtree an odd number of times
-     * <p>
-     * See <a href="https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html">
-     * https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html</a> for more information.
-     */
-    // ported to java from https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html
-    // original code under the BSD license (https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html#License%20to%20Use)
-    //
-    // Copyright (c) 1970-2003, Wm. Randolph Franklin
-    //
-    // Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated 
-    // documentation files (the "Software"), to deal in the Software without restriction, including without limitation 
-    // the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and 
-    // to permit persons to whom the Software is furnished to do so, subject to the following conditions:
-    //
-    // 1. Redistributions of source code must retain the above copyright 
-    //    notice, this list of conditions and the following disclaimers.
-    // 2. Redistributions in binary form must reproduce the above copyright 
-    //    notice in the documentation and/or other materials provided with 
-    //    the distribution.
-    // 3. The name of W. Randolph Franklin may not be used to endorse or 
-    //    promote products derived from this Software without specific 
-    //    prior written permission. 
-    //
-    // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED 
-    // TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL 
-    // THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF 
-    // CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS 
-    // IN THE SOFTWARE. 
-    boolean contains(double latitude, double longitude) {
-      // crossings algorithm is an odd-even algorithm, so we descend the tree xor'ing results along our path
-      boolean res = false;
-      if (latitude <= max) {
-        if (lat1 > latitude != lat2 > latitude) {
-          if (longitude < (lon1 - lon2) * (latitude - lat2) / (lat1 - lat2) + lon2) {
-            res = true;
-          }
-        }
-        if (left != null) {
-          res ^= left.contains(latitude, longitude);
-        }
-        if (right != null && latitude >= low) {
-          res ^= right.contains(latitude, longitude);
-        }
-      }
-      return res;
-    }
-    
-    /** Returns true if the box crosses any edge in this edge subtree */
-    boolean crosses(double minLat, double maxLat, double minLon, double maxLon) {
-      // we just have to cross one edge to answer the question, so we descend the tree and return when we do.
-      if (minLat <= max) {
-        // we compute line intersections of every polygon edge with every box line.
-        // if we find one, return true.
-        // for each box line (AB):
-        //   for each poly line (CD):
-        //     intersects = orient(C,D,A) * orient(C,D,B) <= 0 && orient(A,B,C) * orient(A,B,D) <= 0
-        double cy = lat1;
-        double dy = lat2;
-        double cx = lon1;
-        double dx = lon2;
-        
-        // optimization: see if the rectangle is outside of the "bounding box" of the polyline at all
-        // if not, don't waste our time trying more complicated stuff
-        boolean outside = (cy < minLat && dy < minLat) ||
-                          (cy > maxLat && dy > maxLat) ||
-                          (cx < minLon && dx < minLon) ||
-                          (cx > maxLon && dx > maxLon);
-        if (outside == false) {
-          // does box's top edge intersect polyline?
-          // ax = minLon, bx = maxLon, ay = maxLat, by = maxLat
-          if (orient(cx, cy, dx, dy, minLon, maxLat) * orient(cx, cy, dx, dy, maxLon, maxLat) <= 0 &&
-              orient(minLon, maxLat, maxLon, maxLat, cx, cy) * orient(minLon, maxLat, maxLon, maxLat, dx, dy) <= 0) {
-            return true;
-          }
-
-          // does box's right edge intersect polyline?
-          // ax = maxLon, bx = maxLon, ay = maxLat, by = minLat
-          if (orient(cx, cy, dx, dy, maxLon, maxLat) * orient(cx, cy, dx, dy, maxLon, minLat) <= 0 &&
-              orient(maxLon, maxLat, maxLon, minLat, cx, cy) * orient(maxLon, maxLat, maxLon, minLat, dx, dy) <= 0) {
-            return true;
-          }
-
-          // does box's bottom edge intersect polyline?
-          // ax = maxLon, bx = minLon, ay = minLat, by = minLat
-          if (orient(cx, cy, dx, dy, maxLon, minLat) * orient(cx, cy, dx, dy, minLon, minLat) <= 0 &&
-              orient(maxLon, minLat, minLon, minLat, cx, cy) * orient(maxLon, minLat, minLon, minLat, dx, dy) <= 0) {
-            return true;
-          }
-
-          // does box's left edge intersect polyline?
-          // ax = minLon, bx = minLon, ay = minLat, by = maxLat
-          if (orient(cx, cy, dx, dy, minLon, minLat) * orient(cx, cy, dx, dy, minLon, maxLat) <= 0 &&
-              orient(minLon, minLat, minLon, maxLat, cx, cy) * orient(minLon, minLat, minLon, maxLat, dx, dy) <= 0) {
-            return true;
-          }
-        }
-        
-        if (left != null) {
-          if (left.crosses(minLat, maxLat, minLon, maxLon)) {
-            return true;
-          }
-        }
-        
-        if (right != null && maxLat >= low) {
-          if (right.crosses(minLat, maxLat, minLon, maxLon)) {
-            return true;
-          }
-        }
-      }
-      return false;
-    }
-  }
-
-  /** 
-   * Creates an edge interval tree from a set of polygon vertices.
-   * @return root node of the tree.
-   */
-  private static Edge createTree(double polyLats[], double polyLons[]) {
-    Edge edges[] = new Edge[polyLats.length - 1];
-    for (int i = 1; i < polyLats.length; i++) {
-      double lat1 = polyLats[i-1];
-      double lon1 = polyLons[i-1];
-      double lat2 = polyLats[i];
-      double lon2 = polyLons[i];
-      edges[i - 1] = new Edge(lat1, lon1, lat2, lon2, Math.min(lat1, lat2), Math.max(lat1, lat2));
-    }
-    // sort the edges then build a balanced tree from them
-    Arrays.sort(edges, (left, right) -> {
-      int ret = Double.compare(left.low, right.low);
-      if (ret == 0) {
-        ret = Double.compare(left.max, right.max);
-      }
-      return ret;
-    });
-    return createTree(edges, 0, edges.length - 1);
-  }
-
-  /** Creates tree from sorted edges (with range low and high inclusive) */
-  private static Edge createTree(Edge edges[], int low, int high) {
-    if (low > high) {
-      return null;
-    }
-    // add midpoint
-    int mid = (low + high) >>> 1;
-    Edge newNode = edges[mid];
-    // add children
-    newNode.left = createTree(edges, low, mid - 1);
-    newNode.right = createTree(edges, mid + 1, high);
-    // pull up max values to this node
-    if (newNode.left != null) {
-      newNode.max = Math.max(newNode.max, newNode.left.max);
-    }
-    if (newNode.right != null) {
-      newNode.max = Math.max(newNode.max, newNode.right.max);
-    }
-    return newNode;
-  }
-
-  /**
-   * Returns a positive value if points a, b, and c are arranged in counter-clockwise order,
-   * negative value if clockwise, zero if collinear.
-   */
-  // see the "Orient2D" method described here:
-  // http://www.cs.berkeley.edu/~jrs/meshpapers/robnotes.pdf
-  // https://www.cs.cmu.edu/~quake/robust.html
-  // Note that this one does not yet have the floating point tricks to be exact!
-  private static int orient(double ax, double ay, double bx, double by, double cx, double cy) {
-    double v1 = (bx - ax) * (cy - ay);
-    double v2 = (cx - ax) * (by - ay);
-    if (v1 > v2) {
-      return 1;
-    } else if (v1 < v2) {
-      return -1;
-    } else {
-      return 0;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/837264a4/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonGrid.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonGrid.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonGrid.java
deleted file mode 100644
index 0c185ea..0000000
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonGrid.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.document;
-
-import org.apache.lucene.geo.GeoTestUtil;
-import org.apache.lucene.geo.GeoUtils;
-import org.apache.lucene.geo.Polygon;
-import org.apache.lucene.geo.Rectangle;
-import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.TestUtil;
-
-import static org.apache.lucene.geo.GeoEncodingUtils.decodeLatitude;
-import static org.apache.lucene.geo.GeoEncodingUtils.decodeLongitude;
-import static org.apache.lucene.geo.GeoEncodingUtils.encodeLatitude;
-import static org.apache.lucene.geo.GeoEncodingUtils.encodeLongitude;
-
-/** tests against LatLonGrid (avoiding indexing/queries) */
-public class TestLatLonGrid extends LuceneTestCase {
-
-  /** If the grid returns true, then any point in that cell should return true as well */
-  public void testRandom() throws Exception {
-    for (int i = 0; i < 1000; i++) {
-      Polygon polygon = GeoTestUtil.nextPolygon();
-      Rectangle box = Rectangle.fromPolygon(new Polygon[] { polygon });
-      int minLat = encodeLatitude(box.minLat);
-      int maxLat = encodeLatitude(box.maxLat);
-      int minLon = encodeLongitude(box.minLon);
-      int maxLon = encodeLongitude(box.maxLon);
-      LatLonGrid grid = new LatLonGrid(minLat, maxLat, minLon, maxLon, LatLonTree.build(polygon));
-      // we are in integer space... but exhaustive testing is slow!
-      for (int j = 0; j < 10000; j++) {
-        int lat = TestUtil.nextInt(random(), minLat, maxLat);
-        int lon = TestUtil.nextInt(random(), minLon, maxLon);
-
-        boolean expected = polygon.contains(decodeLatitude(lat),
-                                            decodeLongitude(lon));
-        boolean actual = grid.contains(lat, lon);
-        assertEquals(expected, actual);
-      }
-    }
-  }
-
-  public void testGrowingPolygon() {
-    double centerLat = -80.0 + random().nextDouble() * 160.0;
-    double centerLon = -170.0 + random().nextDouble() * 340.0;
-    double radiusMeters = 0.0;
-    for(int i=0;i<10;i++) {
-      radiusMeters = Math.nextUp(radiusMeters);
-    }
-
-    // Start with a miniscule polygon, and grow it:
-    int gons = TestUtil.nextInt(random(), 4, 10);
-    while (radiusMeters < GeoUtils.EARTH_MEAN_RADIUS_METERS * Math.PI / 2.0 + 1.0) {
-      Polygon polygon;
-      try {
-        polygon = GeoTestUtil.createRegularPolygon(centerLat, centerLon, radiusMeters, gons);
-      } catch (IllegalArgumentException iae) {
-        // OK: we made a too-big poly and it crossed a pole or dateline
-        break;
-      }
-      radiusMeters *= 1.1;
-
-      Rectangle box = Rectangle.fromPolygon(new Polygon[] { polygon });
-      int minLat = encodeLatitude(box.minLat);
-      int maxLat = encodeLatitude(box.maxLat);
-      int minLon = encodeLongitude(box.minLon);
-      int maxLon = encodeLongitude(box.maxLon);
-      LatLonGrid grid = new LatLonGrid(minLat, maxLat, minLon, maxLon, LatLonTree.build(polygon));
-      // we are in integer space... but exhaustive testing is slow!
-      for (int j = 0; j < 1000; j++) {
-        int lat = TestUtil.nextInt(random(), minLat, maxLat);
-        int lon = TestUtil.nextInt(random(), minLon, maxLon);
-
-        boolean expected = polygon.contains(decodeLatitude(lat),
-                                            decodeLongitude(lon));
-        boolean actual = grid.contains(lat, lon);
-        assertEquals(expected, actual);
-      }
-    }
-  }
-  
-  /** create ever-increasing grids and check that too-small polygons don't blow it up */
-  public void testTinyGrids() {
-    double ZERO = decodeLatitude(0);
-    double ONE = decodeLatitude(1);
-    Polygon tiny = new Polygon(new double[] { ZERO, ZERO, ONE, ONE, ZERO }, new double[] { ZERO, ONE, ONE, ZERO, ZERO });
-    for (int max = 1; max < 500000; max++) {
-      LatLonGrid grid = new LatLonGrid(0, max, 0, max, LatLonTree.build(tiny));
-      assertEquals(tiny.contains(decodeLatitude(max), decodeLongitude(max)), grid.contains(max, max));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/837264a4/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonTree.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonTree.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonTree.java
deleted file mode 100644
index c939026..0000000
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonTree.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.document;
-
-import org.apache.lucene.geo.GeoTestUtil;
-import org.apache.lucene.geo.Polygon;
-import org.apache.lucene.geo.Rectangle;
-import org.apache.lucene.index.PointValues.Relation;
-import org.apache.lucene.util.LuceneTestCase;
-
-/** Test LatLonTree against the slower implementation for now */
-public class TestLatLonTree extends LuceneTestCase {
-  
-  /** test that contains() works the same as brute force */
-  public void testContainsRandom() {
-    for (int i = 0; i < 1000; i++) {
-      Polygon polygon = GeoTestUtil.nextPolygon();
-      LatLonTree tree = new LatLonTree(polygon);
-      for (int j = 0; j < 1000; j++) {
-        double point[] = GeoTestUtil.nextPointNear(polygon);
-        boolean expected = polygon.contains(point[0], point[1]);
-        assertEquals(expected, tree.contains(point[0], point[1]));
-      }
-    }
-  }
-  
-  /** test that relate() works the same as brute force */
-  public void testRelateRandom() {
-    for (int i = 0; i < 1000; i++) {
-      Polygon polygon = GeoTestUtil.nextPolygon();
-      LatLonTree tree = new LatLonTree(polygon);
-      for (int j = 0; j < 1000; j++) {
-        Rectangle box = GeoTestUtil.nextBoxNear(polygon);
-        Relation expected = polygon.relate(box.minLat, box.maxLat, box.minLon, box.maxLon);
-        assertEquals(expected, tree.relate(box.minLat, box.maxLat, box.minLon, box.maxLon));
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/837264a4/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointInPolygonQueryImpl.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointInPolygonQueryImpl.java b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointInPolygonQueryImpl.java
index 14b7cc7..047bf27 100644
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointInPolygonQueryImpl.java
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointInPolygonQueryImpl.java
@@ -20,7 +20,7 @@ import java.util.Objects;
 
 import org.apache.lucene.search.MultiTermQuery;
 import org.apache.lucene.spatial.geopoint.document.GeoPointField.TermEncoding;
-import org.apache.lucene.geo.Polygon;
+import org.apache.lucene.geo.Polygon2D;
 import org.apache.lucene.index.PointValues.Relation;
 
 /** Package private implementation for the public facing GeoPointInPolygonQuery delegate class.
@@ -29,13 +29,13 @@ import org.apache.lucene.index.PointValues.Relation;
  */
 final class GeoPointInPolygonQueryImpl extends GeoPointInBBoxQueryImpl {
   private final GeoPointInPolygonQuery polygonQuery;
-  private final Polygon[] polygons;
+  private final Polygon2D polygons;
 
   GeoPointInPolygonQueryImpl(final String field, final TermEncoding termEncoding, final GeoPointInPolygonQuery q,
                              final double minLat, final double maxLat, final double minLon, final double maxLon) {
     super(field, termEncoding, minLat, maxLat, minLon, maxLon);
     this.polygonQuery = Objects.requireNonNull(q);
-    this.polygons = Objects.requireNonNull(q.polygons);
+    this.polygons = Polygon2D.create(q.polygons);
   }
 
   @Override
@@ -59,17 +59,17 @@ final class GeoPointInPolygonQueryImpl extends GeoPointInBBoxQueryImpl {
 
     @Override
     protected boolean cellCrosses(final double minLat, final double maxLat, final double minLon, final double maxLon) {
-      return Polygon.relate(polygons, minLat, maxLat, minLon, maxLon) == Relation.CELL_CROSSES_QUERY;
+      return polygons.relate(minLat, maxLat, minLon, maxLon) == Relation.CELL_CROSSES_QUERY;
     }
 
     @Override
     protected boolean cellWithin(final double minLat, final double maxLat, final double minLon, final double maxLon) {
-      return Polygon.relate(polygons, minLat, maxLat, minLon, maxLon) == Relation.CELL_INSIDE_QUERY;
+      return polygons.relate(minLat, maxLat, minLon, maxLon) == Relation.CELL_INSIDE_QUERY;
     }
 
     @Override
     protected boolean cellIntersectsShape(final double minLat, final double maxLat, final double minLon, final double maxLon) {
-      return Polygon.relate(polygons, minLat, maxLat, minLon, maxLon) != Relation.CELL_OUTSIDE_QUERY;
+      return polygons.relate(minLat, maxLat, minLon, maxLon) != Relation.CELL_OUTSIDE_QUERY;
     }
 
     /**
@@ -77,11 +77,11 @@ final class GeoPointInPolygonQueryImpl extends GeoPointInBBoxQueryImpl {
      * {@link org.apache.lucene.spatial.geopoint.search.GeoPointTermsEnum#accept} method is called to match
      * encoded terms that fall within the bounding box of the polygon. Those documents that pass the initial
      * bounding box filter are then compared to the provided polygon using the
-     * {@link Polygon#contains(Polygon[], double, double)} method.
+     * {@link Polygon2D#contains(double, double)} method.
      */
     @Override
     protected boolean postFilter(final double lat, final double lon) {
-      return Polygon.contains(polygons, lat, lon);
+      return polygons.contains(lat, lon);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/837264a4/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 6bc1e6e..bda4cde 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
@@ -1112,7 +1112,7 @@ public abstract class BaseGeoPointTestCase extends LuceneTestCase {
         } else if (Double.isNaN(lats[id])) {
           expected = false;
         } else {
-          expected = polygon.contains(lats[id], lons[id]);
+          expected = GeoTestUtil.containsSlowly(polygon, lats[id], lons[id]);
         }
 
         if (hits.get(docID) != expected) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/837264a4/lucene/test-framework/src/java/org/apache/lucene/geo/GeoTestUtil.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/geo/GeoTestUtil.java b/lucene/test-framework/src/java/org/apache/lucene/geo/GeoTestUtil.java
index 98e2966..62b824f 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/geo/GeoTestUtil.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/geo/GeoTestUtil.java
@@ -642,4 +642,58 @@ public class GeoTestUtil {
     sb.append("</svg>\n");
     return sb.toString();
   }
+
+  /**
+   * Simple slow point in polygon check (for testing)
+   */
+  // direct port of PNPOLY C code (https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html)
+  // this allows us to improve the code yet still ensure we have its properties
+  // it is under the BSD license (https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html#License%20to%20Use)
+  //
+  // Copyright (c) 1970-2003, Wm. Randolph Franklin
+  //
+  // Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated 
+  // documentation files (the "Software"), to deal in the Software without restriction, including without limitation 
+  // the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and 
+  // to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+  //
+  // 1. Redistributions of source code must retain the above copyright 
+  //    notice, this list of conditions and the following disclaimers.
+  // 2. Redistributions in binary form must reproduce the above copyright 
+  //    notice in the documentation and/or other materials provided with 
+  //    the distribution.
+  // 3. The name of W. Randolph Franklin may not be used to endorse or 
+  //    promote products derived from this Software without specific 
+  //    prior written permission. 
+  //
+  // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED 
+  // TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL 
+  // THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF 
+  // CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS 
+  // IN THE SOFTWARE. 
+  public static boolean containsSlowly(Polygon polygon, double latitude, double longitude) {
+    if (polygon.getHoles().length > 0) {
+      throw new UnsupportedOperationException("this testing method does not support holes");
+    }
+    double polyLats[] = polygon.getPolyLats();
+    double polyLons[] = polygon.getPolyLons();
+    // bounding box check required due to rounding errors (we don't solve that problem)
+    if (latitude < polygon.minLat || latitude > polygon.maxLat || longitude < polygon.minLon || longitude > polygon.maxLon) {
+      return false;
+    }
+    
+    boolean c = false;
+    int i, j;
+    int nvert = polyLats.length;
+    double verty[] = polyLats;
+    double vertx[] = polyLons;
+    double testy = latitude;
+    double testx = longitude;
+    for (i = 0, j = nvert-1; i < nvert; j = i++) {
+      if ( ((verty[i]>testy) != (verty[j]>testy)) &&
+     (testx < (vertx[j]-vertx[i]) * (testy-verty[i]) / (verty[j]-verty[i]) + vertx[i]) )
+         c = !c;
+    }
+    return c;
+  }
 }


[50/50] [abbrv] lucene-solr:solr-5750: SOLR-5750: refactor SnapShooter.createSnapshot surrounding logic Fixed a bug too; createSnapshot() wasn't reserving the commit with the deletion policy

Posted by ds...@apache.org.
SOLR-5750: refactor SnapShooter.createSnapshot surrounding logic
Fixed a bug too; createSnapshot() wasn't reserving the commit with the deletion policy


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

Branch: refs/heads/solr-5750
Commit: ca173ab1af9824ba262ba206bcaab9babbb2ac89
Parents: e54718d
Author: David Smiley <ds...@apache.org>
Authored: Wed Apr 27 14:20:48 2016 -0400
Committer: David Smiley <ds...@apache.org>
Committed: Wed Apr 27 14:20:48 2016 -0400

----------------------------------------------------------------------
 .../apache/solr/handler/ReplicationHandler.java |  7 +-
 .../org/apache/solr/handler/SnapShooter.java    | 97 +++++++++-----------
 .../solr/handler/admin/CoreAdminOperation.java  | 32 ++-----
 3 files changed, 53 insertions(+), 83 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ca173ab1/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 c98fabf..4dbcef2 100644
--- a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
@@ -87,7 +87,6 @@ import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.update.CdcrUpdateLog;
 import org.apache.solr.update.SolrIndexWriter;
-import org.apache.solr.update.UpdateLog;
 import org.apache.solr.update.VersionInfo;
 import org.apache.solr.util.DefaultSolrThreadFactory;
 import org.apache.solr.util.NumberUtils;
@@ -195,7 +194,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
 
   volatile IndexCommit indexCommitPoint;
 
-  volatile NamedList<Object> snapShootDetails;
+  volatile NamedList<?> snapShootDetails;
 
   private AtomicBoolean replicationEnabled = new AtomicBoolean(true);
 
@@ -509,7 +508,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
       // small race here before the commit point is saved
       SnapShooter snapShooter = new SnapShooter(core, params.get("location"), params.get(NAME));
       snapShooter.validateCreateSnapshot();
-      snapShooter.createSnapAsync(indexCommit, numberToKeep, this);
+      snapShooter.createSnapAsync(indexCommit, numberToKeep, (nl) -> snapShootDetails = nl);
 
     } catch (Exception e) {
       LOG.warn("Exception during creating a snapshot", e);
@@ -1323,7 +1322,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
             }
             SnapShooter snapShooter = new SnapShooter(core, null, null);
             snapShooter.validateCreateSnapshot();
-            snapShooter.createSnapAsync(currentCommitPoint, numberToKeep, ReplicationHandler.this);
+            snapShooter.createSnapAsync(currentCommitPoint, numberToKeep, (nl) -> snapShootDetails = nl);
           } catch (Exception e) {
             LOG.error("Exception while snapshooting", e);
           }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ca173ab1/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 2e20b72..92b9246 100644
--- a/solr/core/src/java/org/apache/solr/handler/SnapShooter.java
+++ b/solr/core/src/java/org/apache/solr/handler/SnapShooter.java
@@ -26,6 +26,7 @@ import java.util.Collections;
 import java.util.Date;
 import java.util.List;
 import java.util.Locale;
+import java.util.function.Consumer;
 
 import org.apache.lucene.index.IndexCommit;
 import org.apache.lucene.store.Directory;
@@ -36,6 +37,7 @@ import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.NamedList;
 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.search.SolrIndexSearcher;
 import org.apache.solr.util.RefCounted;
@@ -75,21 +77,6 @@ public class SnapShooter {
     }
   }
 
-  void createSnapAsync(final IndexCommit indexCommit, final int numberToKeep, final ReplicationHandler replicationHandler) {
-    solrCore.getDeletionPolicy().saveCommitPoint(indexCommit.getGeneration());
-
-    new Thread() {
-      @Override
-      public void run() {
-        if(snapshotName != null) {
-          createSnapshot(indexCommit, replicationHandler);
-        } else {
-          createSnapshot(indexCommit, replicationHandler);
-          deleteOldBackups(numberToKeep);
-        }
-      }
-    }.start();
-  }
 
   public void validateDeleteSnapshot() {
     boolean dirFound = false;
@@ -126,50 +113,55 @@ public class SnapShooter {
     }
   }
 
-  //nocommit - copy pasted from createSnapshot. Need to reconcile tho
-  public NamedList createSnapshot() {
+  public NamedList createSnapshot() throws Exception {
+    IndexDeletionPolicyWrapper deletionPolicy = solrCore.getDeletionPolicy();
     RefCounted<SolrIndexSearcher> searcher = solrCore.getSearcher();
-    IndexCommit indexCommit = null;
-
-    NamedList<Object> details = new NamedList<>();
     try {
-      details.add("startTime", new Date().toString());
-      LOG.info("Creating backup snapshot " + (snapshotName == null ? "<not named>" : snapshotName) + " at " + snapDir);
-
-      indexCommit = searcher.get().getIndexReader().getIndexCommit();
-      Collection<String> files = indexCommit.getFileNames();
-
-      Directory dir = solrCore.getDirectoryFactory().get(solrCore.getIndexDir(), DirContext.DEFAULT, solrCore.getSolrConfig().indexConfig.lockType);
+      //TODO should we try solrCore.getDeletionPolicy().getLatestCommit() first?
+      IndexCommit indexCommit = searcher.get().getIndexReader().getIndexCommit();
+      deletionPolicy.saveCommitPoint(indexCommit.getGeneration());
       try {
-        copyFiles(dir, files, snapShotDir);
+        return createSnapshot(indexCommit);
       } finally {
-        solrCore.getDirectoryFactory().release(dir);
+        deletionPolicy.releaseCommitPoint(indexCommit.getGeneration());
       }
-
-      details.add("fileCount", files.size());
-      details.add("status", "success");
-      details.add("snapshotCompletedAt", new Date().toString());
-      details.add("snapshotName", snapshotName);
-      LOG.info("Done creating backup snapshot: " + (snapshotName == null ? "<not named>" : snapshotName) +
-          " at " + snapDir);
-    } catch (Exception e) {
-      IndexFetcher.delTree(snapShotDir);
-      LOG.error("Exception while creating snapshot", e);
-      details.add("snapShootException", e.getMessage());
     } finally {
-      solrCore.getDeletionPolicy().releaseCommitPoint(indexCommit.getGeneration());
       searcher.decref();
     }
-    return details;
   }
 
-  void createSnapshot(final IndexCommit indexCommit, ReplicationHandler replicationHandler) {
+  public void createSnapAsync(final IndexCommit indexCommit, final int numberToKeep, Consumer<NamedList> result) {
+    solrCore.getDeletionPolicy().saveCommitPoint(indexCommit.getGeneration());
+
+    new Thread() { //TODO should use Solr's ExecutorUtil
+      @Override
+      public void run() {
+        try {
+          result.accept(createSnapshot(indexCommit));
+        } catch (Exception e) {
+          LOG.error("Exception while creating snapshot", e);
+          NamedList snapShootDetails = new NamedList<>();
+          snapShootDetails.add("snapShootException", e.getMessage());
+          result.accept(snapShootDetails);
+        } finally {
+          solrCore.getDeletionPolicy().releaseCommitPoint(indexCommit.getGeneration());
+        }
+        if (snapshotName == null) {
+          deleteOldBackups(numberToKeep);
+        }
+      }
+    }.start();
+  }
+
+  // note: remember to reserve the indexCommit first so it won't get deleted concurrently
+  protected NamedList createSnapshot(final IndexCommit indexCommit) throws Exception {
     LOG.info("Creating backup snapshot " + (snapshotName == null ? "<not named>" : snapshotName) + " at " + snapDir);
-    NamedList<Object> details = new NamedList<>();
-    details.add("startTime", new Date().toString());
+    boolean success = false;
     try {
-      Collection<String> files = indexCommit.getFileNames();
+      NamedList<Object> details = new NamedList<>();
+      details.add("startTime", new Date().toString());//bad; should be Instant.now().toString()
 
+      Collection<String> files = indexCommit.getFileNames();
       Directory dir = solrCore.getDirectoryFactory().get(solrCore.getIndexDir(), DirContext.DEFAULT, solrCore.getSolrConfig().indexConfig.lockType);
       try {
         copyFiles(dir, files, snapShotDir);
@@ -179,17 +171,16 @@ public class SnapShooter {
 
       details.add("fileCount", files.size());
       details.add("status", "success");
-      details.add("snapshotCompletedAt", new Date().toString());
+      details.add("snapshotCompletedAt", new Date().toString());//bad; should be Instant.now().toString()
       details.add("snapshotName", snapshotName);
       LOG.info("Done creating backup snapshot: " + (snapshotName == null ? "<not named>" : snapshotName) +
           " at " + snapDir);
-    } catch (Exception e) {
-      IndexFetcher.delTree(snapShotDir);
-      LOG.error("Exception while creating snapshot", e);
-      details.add("snapShootException", e.getMessage());
+      success = true;
+      return details;
     } finally {
-      solrCore.getDeletionPolicy().releaseCommitPoint(indexCommit.getGeneration());
-      replicationHandler.snapShootDetails = details;
+      if (!success) {
+        IndexFetcher.delTree(snapShotDir);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ca173ab1/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 2365b87..6cf4589 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
@@ -39,7 +39,6 @@ import org.apache.lucene.util.IOUtils;
 import org.apache.solr.cloud.CloudDescriptor;
 import org.apache.solr.cloud.SyncStrategy;
 import org.apache.solr.cloud.ZkController;
-import org.apache.solr.common.NonExistentCoreException;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
@@ -82,26 +81,7 @@ import org.slf4j.LoggerFactory;
 import static org.apache.solr.common.cloud.DocCollection.DOC_ROUTER;
 import static org.apache.solr.common.params.CommonParams.NAME;
 import static org.apache.solr.common.params.CommonParams.PATH;
-import static org.apache.solr.common.params.CoreAdminParams.CoreAdminAction.BACKUPCORE;
-import static org.apache.solr.common.params.CoreAdminParams.CoreAdminAction.CREATE;
-import static org.apache.solr.common.params.CoreAdminParams.CoreAdminAction.FORCEPREPAREFORLEADERSHIP;
-import static org.apache.solr.common.params.CoreAdminParams.CoreAdminAction.INVOKE;
-import static org.apache.solr.common.params.CoreAdminParams.CoreAdminAction.MERGEINDEXES;
-import static org.apache.solr.common.params.CoreAdminParams.CoreAdminAction.OVERSEEROP;
-import static org.apache.solr.common.params.CoreAdminParams.CoreAdminAction.PREPRECOVERY;
-import static org.apache.solr.common.params.CoreAdminParams.CoreAdminAction.REJOINLEADERELECTION;
-import static org.apache.solr.common.params.CoreAdminParams.CoreAdminAction.RELOAD;
-import static org.apache.solr.common.params.CoreAdminParams.CoreAdminAction.RENAME;
-import static org.apache.solr.common.params.CoreAdminParams.CoreAdminAction.REQUESTAPPLYUPDATES;
-import static org.apache.solr.common.params.CoreAdminParams.CoreAdminAction.REQUESTBUFFERUPDATES;
-import static org.apache.solr.common.params.CoreAdminParams.CoreAdminAction.REQUESTRECOVERY;
-import static org.apache.solr.common.params.CoreAdminParams.CoreAdminAction.REQUESTSTATUS;
-import static org.apache.solr.common.params.CoreAdminParams.CoreAdminAction.REQUESTSYNCSHARD;
-import static org.apache.solr.common.params.CoreAdminParams.CoreAdminAction.RESTORECORE;
-import static org.apache.solr.common.params.CoreAdminParams.CoreAdminAction.SPLIT;
-import static org.apache.solr.common.params.CoreAdminParams.CoreAdminAction.STATUS;
-import static org.apache.solr.common.params.CoreAdminParams.CoreAdminAction.SWAP;
-import static org.apache.solr.common.params.CoreAdminParams.CoreAdminAction.UNLOAD;
+import static org.apache.solr.common.params.CoreAdminParams.CoreAdminAction.*;
 import static org.apache.solr.handler.admin.CoreAdminHandler.COMPLETED;
 import static org.apache.solr.handler.admin.CoreAdminHandler.CallInfo;
 import static org.apache.solr.handler.admin.CoreAdminHandler.FAILED;
@@ -885,10 +865,10 @@ enum CoreAdminOperation {
       try (SolrCore core = callInfo.handler.coreContainer.getCore(cname)) {
         SnapShooter snapShooter = new SnapShooter(core, location, name);
         snapShooter.validateCreateSnapshot();
-        NamedList details = snapShooter.createSnapshot();
-        if (details.get("snapShootException") != null) {
-          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Failed to backup core=" + core.getName());
-        }
+        snapShooter.createSnapshot();
+      } catch (Exception e) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+            "Failed to backup core=" + cname + " because " + e, e);
       }
     }
   },
@@ -912,7 +892,7 @@ enum CoreAdminOperation {
       }
 
       String location = params.get("location");
-      if (name == null) {
+      if (location == null) {
         throw new IllegalArgumentException("location is required");
       }
 


[35/50] [abbrv] lucene-solr:solr-5750: Make the static CoreParser.parseXML method package-private instead of private so that it's visible to and can be used by tests.

Posted by ds...@apache.org.
Make the static CoreParser.parseXML method package-private instead of private so that it's visible to and can be used by tests.


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

Branch: refs/heads/solr-5750
Commit: 9d8fad7a6361bffe360fb40eaf29abd5a6105c4e
Parents: 506d61a
Author: Christine Poerschke <cp...@apache.org>
Authored: Tue Apr 26 15:10:58 2016 +0100
Committer: Christine Poerschke <cp...@apache.org>
Committed: Tue Apr 26 15:10:58 2016 +0100

----------------------------------------------------------------------
 .../src/java/org/apache/lucene/queryparser/xml/CoreParser.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9d8fad7a/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/CoreParser.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/CoreParser.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/CoreParser.java
index 1416f25..0c5cf84 100644
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/CoreParser.java
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/CoreParser.java
@@ -116,7 +116,7 @@ public class CoreParser implements QueryBuilder {
     queryFactory.addBuilder(nodeName, builder);
   }
 
-  private static Document parseXML(InputStream pXmlFile) throws ParserException {
+  static Document parseXML(InputStream pXmlFile) throws ParserException {
     DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
     DocumentBuilder db = null;
     try {


[19/50] [abbrv] lucene-solr:solr-5750: LUCENE-7244: Complain if the holes are outside the polygon.

Posted by ds...@apache.org.
LUCENE-7244: Complain if the holes are outside the polygon.


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

Branch: refs/heads/solr-5750
Commit: 38ebd906e830e793d7df364163f0baab049ffa47
Parents: 88c9da6
Author: Karl Wright <Da...@gmail.com>
Authored: Fri Apr 22 16:35:51 2016 -0400
Committer: Karl Wright <Da...@gmail.com>
Committed: Fri Apr 22 16:37:31 2016 -0400

----------------------------------------------------------------------
 .../spatial3d/geom/GeoConcavePolygon.java       | 35 +++++++++++---------
 .../lucene/spatial3d/geom/GeoConvexPolygon.java | 35 +++++++++++---------
 2 files changed, 40 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/38ebd906/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoConcavePolygon.java
----------------------------------------------------------------------
diff --git a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoConcavePolygon.java b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoConcavePolygon.java
index 124b46b..8eaea1a 100644
--- a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoConcavePolygon.java
+++ b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoConcavePolygon.java
@@ -207,21 +207,6 @@ class GeoConcavePolygon extends GeoBasePolygon {
       invertedEdges[i] = new SidedPlane(edges[i]);
       notableEdgePoints[i] = new GeoPoint[]{start, end};
     }
-    /* Disable since GeoPolygonFactory does this too.
-    // In order to naively confirm that the polygon is concave, I would need to
-    // check every edge, and verify that every point (other than the edge endpoints)
-    // is within the edge's sided plane.  This is an order n^2 operation.  That's still
-    // not wrong, though, because everything else about polygons has a similar cost.
-    for (int edgeIndex = 0; edgeIndex < edges.length; edgeIndex++) {
-      final SidedPlane edge = edges[edgeIndex];
-      for (int pointIndex = 0; pointIndex < points.size(); pointIndex++) {
-        if (pointIndex != edgeIndex && pointIndex != legalIndex(edgeIndex + 1)) {
-          if (edge.isWithin(points.get(pointIndex)))
-            throw new IllegalArgumentException("Polygon is not concave: Point " + points.get(pointIndex) + " Edge " + edge);
-        }
-      }
-    }
-    */
     
     // For each edge, create a bounds object.
     eitherBounds = new HashMap<>(edges.length);
@@ -241,6 +226,26 @@ class GeoConcavePolygon extends GeoBasePolygon {
 
     // Pick an edge point arbitrarily
     edgePoints = new GeoPoint[]{points.get(0)};
+
+    if (isWithinHoles(points.get(0))) {
+      throw new IllegalArgumentException("Polygon edge intersects a polygon hole; not allowed");
+    }
+
+  }
+
+  /** Check if a point is within the provided holes.
+   *@param point point to check.
+   *@return true if the point is within any of the holes.
+   */
+  protected boolean isWithinHoles(final GeoPoint point) {
+    if (holes != null) {
+      for (final GeoPolygon hole : holes) {
+        if (hole.isWithin(point)) {
+          return true;
+        }
+      }
+    }
+    return false;
   }
 
   /** Compute a legal point index from a possibly illegal one, that may have wrapped.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/38ebd906/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoConvexPolygon.java
----------------------------------------------------------------------
diff --git a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoConvexPolygon.java b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoConvexPolygon.java
index 64aa7c4..17a2120 100755
--- a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoConvexPolygon.java
+++ b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoConvexPolygon.java
@@ -203,21 +203,6 @@ class GeoConvexPolygon extends GeoBasePolygon {
       edges[i] = sp;
       notableEdgePoints[i] = new GeoPoint[]{start, end};
     }
-    /* Disabled since GeoPolygonFactory does the checking too.
-    // In order to naively confirm that the polygon is convex, I would need to
-    // check every edge, and verify that every point (other than the edge endpoints)
-    // is within the edge's sided plane.  This is an order n^2 operation.  That's still
-    // not wrong, though, because everything else about polygons has a similar cost.
-    for (int edgeIndex = 0; edgeIndex < edges.length; edgeIndex++) {
-      final SidedPlane edge = edges[edgeIndex];
-      for (int pointIndex = 0; pointIndex < points.size(); pointIndex++) {
-        if (pointIndex != edgeIndex && pointIndex != legalIndex(edgeIndex + 1)) {
-          if (!edge.isWithin(points.get(pointIndex)))
-            throw new IllegalArgumentException("Polygon is not convex: Point " + points.get(pointIndex) + " Edge " + edge);
-        }
-      }
-    }
-    */
     
     // For each edge, create a bounds object.
     eitherBounds = new HashMap<>(edges.length);
@@ -236,8 +221,28 @@ class GeoConvexPolygon extends GeoBasePolygon {
     
     // Pick an edge point arbitrarily
     edgePoints = new GeoPoint[]{points.get(0)};
+    
+    if (isWithinHoles(points.get(0))) {
+      throw new IllegalArgumentException("Polygon edge intersects a polygon hole; not allowed");
+    }
+
   }
 
+  /** Check if a point is within the provided holes.
+   *@param point point to check.
+   *@return true if the point is within any of the holes.
+   */
+  protected boolean isWithinHoles(final GeoPoint point) {
+    if (holes != null) {
+      for (final GeoPolygon hole : holes) {
+        if (hole.isWithin(point)) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+  
   /** Compute a legal point index from a possibly illegal one, that may have wrapped.
    *@param index is the index.
    *@return the normalized index.


[47/50] [abbrv] lucene-solr:solr-5750: SOLR-5750: remove nocommit for snapshotName prop name; keep consistent

Posted by ds...@apache.org.
SOLR-5750: remove nocommit for snapshotName prop name; keep consistent


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

Branch: refs/heads/solr-5750
Commit: 42b6ed4909f0c7e544b294fcdf98fed626090f29
Parents: 6a9cc1a
Author: David Smiley <ds...@apache.org>
Authored: Wed Apr 27 11:39:03 2016 -0400
Committer: David Smiley <ds...@apache.org>
Committed: Wed Apr 27 11:39:03 2016 -0400

----------------------------------------------------------------------
 .../cloud/OverseerCollectionMessageHandler.java  | 19 ++++++++++---------
 .../solrj/request/CollectionAdminRequest.java    |  4 ++--
 2 files changed, 12 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/42b6ed49/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java b/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
index 40907f5..e1ea1dc 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
@@ -492,12 +492,13 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
       }
     }
 
-    log.info("Completed restoring collection={} backup={}", restoreCollection, backupName);
+    log.info("Completed restoring collection={} snapshotName={}", restoreCollection, backupName);
   }
 
+  //nocommit this should precede processRestoreAction
   private void processBackupAction(ZkNodeProps message, NamedList results) throws IOException, KeeperException, InterruptedException {
     String collectionName =  message.getStr(COLLECTION_PROP);
-    String name =  message.getStr(NAME);
+    String backupName =  message.getStr(NAME);
     String location = message.getStr("location");
     ShardHandler shardHandler = shardHandlerFactory.getShardHandler();
     String asyncId = message.getStr(ASYNC);
@@ -505,7 +506,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
     Instant startTime = Instant.now();
 
     // note: we assume a shared files system to backup a collection, since a collection is distributed
-    Path backupPath = Paths.get(location).resolve(name).toAbsolutePath();
+    Path backupPath = Paths.get(location).resolve(backupName).toAbsolutePath();
 
     //Validating if the directory already exists.
     if (Files.exists(backupPath)) {
@@ -514,7 +515,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
     }
     Files.createDirectory(backupPath); // create now
 
-    log.info("Starting backup of collection={} with backup_name={} at location={}", collectionName, name,
+    log.info("Starting backup of collection={} with snapshotName={} at location={}", collectionName, backupName,
         backupPath);
 
     for (Slice slice : zkStateReader.getClusterState().getCollection(collectionName).getActiveSlices()) {
@@ -529,13 +530,13 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
       params.set(CORE_NAME_PROP, coreName);
 
       sendShardRequest(replica.getNodeName(), params, shardHandler, asyncId, requestMap);
-      log.debug("Sent backup request to core={} for backup_name={}", coreName, name);
+      log.debug("Sent backup request to core={} for snapshotName={}", coreName, backupName);
     }
-    log.debug("Sent backup requests to all shard leaders for backup_name={}", name);
+    log.debug("Sent backup requests to all shard leaders for snapshotName={}", backupName);
 
     processResponses(results, shardHandler, true, "Could not backup all replicas", asyncId, requestMap);
 
-    log.info("Starting to backup ZK data for backup_name={}", name);
+    log.info("Starting to backup ZK data for snapshotName={}", backupName);
 
     //Download the configs
     String configName = zkStateReader.readConfigName(collectionName);
@@ -552,7 +553,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
     Path propertiesPath = zkBackup.resolve("backup.properties");
     Properties properties = new Properties();
 
-    properties.put("snapshotName", name); //nocommit or simply "backupName" (consistent with "backup" term?)
+    properties.put("snapshotName", backupName);
     properties.put("collection", collectionName);
     properties.put("collection.configName", configName);
     properties.put("startTime", startTime.toString());
@@ -564,7 +565,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
       properties.store(os, "Snapshot properties file");
     }
 
-    log.info("Completed backing up ZK data for backup={}", name);
+    log.info("Completed backing up ZK data for snapshotName={}", backupName);
   }
 
   @SuppressWarnings("unchecked")

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/42b6ed49/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
index 4af5c00..4a95133 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
@@ -568,7 +568,7 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
     public SolrParams getParams() {
       ModifiableSolrParams params = (ModifiableSolrParams) super.getParams();
       params.set(CoreAdminParams.COLLECTION, collection); // nocommit reconsider to use NAME (set via super.getParams)
-      params.set(CoreAdminParams.NAME, name); // nocommit reconsider to use backupName?
+      params.set(CoreAdminParams.NAME, name); // nocommit reconsider to use snapshotName or snapshot?
       params.set("location", location); //note: optional
       return params;
     }
@@ -641,7 +641,7 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
     public SolrParams getParams() {
       ModifiableSolrParams params = (ModifiableSolrParams) super.getParams();
       params.set(CoreAdminParams.COLLECTION, collection); // nocommit reconsider to use NAME (set via super.getParams)
-      params.set(CoreAdminParams.NAME, backupName); // nocommit reconsider to use backupName?
+      params.set(CoreAdminParams.NAME, backupName); // nocommit reconsider to use snapshotName or snapshot?
       params.set("location", location); //note: optional
       params.set("collection.configName", configName); //note: optional
       if (maxShardsPerNode != null) {


[31/50] [abbrv] lucene-solr:solr-5750: SOLR-9029: fix rare ZkStateReader visibility race during collection state format update

Posted by ds...@apache.org.
SOLR-9029: fix rare ZkStateReader visibility race during collection state format update


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

Branch: refs/heads/solr-5750
Commit: 89c65af2a6e5f1c8216c1202f65e8d670ef14385
Parents: 922265b
Author: Scott Blum <dr...@apache.org>
Authored: Mon Apr 25 21:15:02 2016 -0400
Committer: Scott Blum <dr...@apache.org>
Committed: Mon Apr 25 21:15:02 2016 -0400

----------------------------------------------------------------------
 solr/CHANGES.txt                                            | 2 ++
 .../java/org/apache/solr/common/cloud/ZkStateReader.java    | 9 +++------
 2 files changed, 5 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89c65af2/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 999bd73..7702950 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -150,6 +150,8 @@ Bug Fixes
 
 * SOLR-8992: Restore Schema API GET method functionality removed in 6.0 (noble, Steve Rowe)
 
+* SOLR-9029: fix rare ZkStateReader visibility race during collection state format update (Scott Blum, hossman)
+
 Optimizations
 ----------------------
 * SOLR-8722: Don't force a full ZkStateReader refresh on every Overseer operation.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89c65af2/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 568c791..1e57d7e 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
@@ -263,9 +263,9 @@ public class ZkStateReader implements Closeable {
       }
 
       ClusterState.CollectionRef ref = clusterState.getCollectionRef(collection);
-      if (ref == null) {
-        // We don't know anything about this collection, maybe it's new?
-        // First try to update the legacy cluster state.
+      if (ref == null || legacyCollectionStates.containsKey(collection)) {
+        // We either don't know anything about this collection (maybe it's new?) or it's legacy.
+        // First update the legacy cluster state.
         refreshLegacyClusterState(null);
         if (!legacyCollectionStates.containsKey(collection)) {
           // No dice, see if a new collection just got created.
@@ -283,9 +283,6 @@ public class ZkStateReader implements Closeable {
         }
         // Edge case: if there's no external collection, try refreshing legacy cluster state in case it's there.
         refreshLegacyClusterState(null);
-      } else if (legacyCollectionStates.containsKey(collection)) {
-        // Exists, and lives in legacy cluster state, force a refresh.
-        refreshLegacyClusterState(null);
       } else if (watchedCollectionStates.containsKey(collection)) {
         // Exists as a watched collection, force a refresh.
         DocCollection newState = fetchCollectionState(collection, null);


[33/50] [abbrv] lucene-solr:solr-5750: SOLR-9041: 'core-admin-read' and 'core-admin-edit' are well known permissions

Posted by ds...@apache.org.
SOLR-9041: 'core-admin-read' and 'core-admin-edit' are well known permissions


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

Branch: refs/heads/solr-5750
Commit: 0233765c274ec9d308ca4eb621f26bd4ed9d0a39
Parents: 7b89d0f
Author: Noble Paul <no...@apache.org>
Authored: Tue Apr 26 19:24:23 2016 +0530
Committer: Noble Paul <no...@apache.org>
Committed: Tue Apr 26 19:24:23 2016 +0530

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  2 +
 .../solr/handler/admin/CollectionsHandler.java  |  2 +-
 .../solr/handler/admin/CoreAdminHandler.java    | 18 ++++++++-
 .../solr/security/PermissionNameProvider.java   |  2 +
 .../TestRuleBasedAuthorizationPlugin.java       | 40 ++++++++++++++++++++
 .../solr/common/params/CoreAdminParams.java     | 16 ++++++--
 6 files changed, 75 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0233765c/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 6034851..905c38d 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -111,6 +111,8 @@ New Features
 
 * SOLR-9020: Implement StatementImpl/ResultSetImpl get/set fetch* methods and proper errors for traversal methods (Kevin Risden)
 
+* SOLR-9041: 'core-admin-read' and 'core-admin-edit' are well known permissions (noble)
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0233765c/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
index 64b10ab..29a978f 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
@@ -151,7 +151,7 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
   @Override
   public PermissionNameProvider.Name getPermissionName(AuthorizationContext ctx) {
     String action = ctx.getParams().get("action");
-    if (action == null) return null;
+    if (action == null) return PermissionNameProvider.Name.COLL_READ_PERM;
     CollectionParams.CollectionAction collectionAction = CollectionParams.CollectionAction.get(action);
     if (collectionAction == null) return null;
     return collectionAction.isWrite ?

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0233765c/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 9eeef10..6e295f7 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
@@ -33,6 +33,7 @@ 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.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.params.CommonAdminParams;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.params.CoreAdminParams.CoreAdminAction;
@@ -45,6 +46,8 @@ import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.handler.RequestHandlerBase;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.security.AuthorizationContext;
+import org.apache.solr.security.PermissionNameProvider;
 import org.apache.solr.util.DefaultSolrThreadFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -52,12 +55,14 @@ import org.slf4j.MDC;
 
 import static org.apache.solr.common.params.CoreAdminParams.ACTION;
 import static org.apache.solr.common.params.CoreAdminParams.CoreAdminAction.STATUS;
+import static org.apache.solr.security.PermissionNameProvider.Name.CORE_EDIT_PERM;
+import static org.apache.solr.security.PermissionNameProvider.Name.CORE_READ_PERM;
 
 /**
  *
  * @since solr 1.3
  */
-public class CoreAdminHandler extends RequestHandlerBase {
+public class CoreAdminHandler extends RequestHandlerBase implements PermissionNameProvider {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   protected final CoreContainer coreContainer;
   protected final Map<String, Map<String, TaskObject>> requestStatusMap;
@@ -262,6 +267,17 @@ public class CoreAdminHandler extends RequestHandlerBase {
     return "Manage Multiple Solr Cores";
   }
 
+  @Override
+  public Name getPermissionName(AuthorizationContext ctx) {
+    String action = ctx.getParams().get(CoreAdminParams.ACTION);
+    if (action == null) return CORE_READ_PERM;
+    CoreAdminParams.CoreAdminAction coreAction = CoreAdminParams.CoreAdminAction.get(action);
+    if (coreAction == null) return CORE_READ_PERM;
+    return coreAction.isRead ?
+        CORE_READ_PERM :
+        CORE_EDIT_PERM;
+  }
+
   /**
    * Helper class to manage the tasks to be tracked.
    * This contains the taskId, request and the response (if available).

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0233765c/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java b/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java
index 2dec433..4b6af47 100644
--- a/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java
+++ b/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java
@@ -36,6 +36,8 @@ public interface PermissionNameProvider {
   enum Name {
     COLL_EDIT_PERM("collection-admin-edit", null),
     COLL_READ_PERM("collection-admin-read", null),
+    CORE_READ_PERM("core-admin-read", null),
+    CORE_EDIT_PERM("core-admin-edit", null),
     READ_PERM("read", "*"),
     UPDATE_PERM("update", "*"),
     CONFIG_EDIT_PERM("config-edit", "*"),

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0233765c/solr/core/src/test/org/apache/solr/security/TestRuleBasedAuthorizationPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/TestRuleBasedAuthorizationPlugin.java b/solr/core/src/test/org/apache/solr/security/TestRuleBasedAuthorizationPlugin.java
index 7a92f8f..5abf21c 100644
--- a/solr/core/src/test/org/apache/solr/security/TestRuleBasedAuthorizationPlugin.java
+++ b/solr/core/src/test/org/apache/solr/security/TestRuleBasedAuthorizationPlugin.java
@@ -35,6 +35,7 @@ import org.apache.solr.handler.ReplicationHandler;
 import org.apache.solr.handler.SchemaHandler;
 import org.apache.solr.handler.UpdateRequestHandler;
 import org.apache.solr.handler.admin.CollectionsHandler;
+import org.apache.solr.handler.admin.CoreAdminHandler;
 import org.apache.solr.handler.component.SearchHandler;
 import org.apache.solr.security.AuthorizationContext.CollectionRequest;
 import org.apache.solr.security.AuthorizationContext.RequestType;
@@ -211,6 +212,45 @@ public class TestRuleBasedAuthorizationPlugin extends SolrTestCaseJ4 {
         "params", new MapSolrParams(singletonMap("action", "CREATE")))
         , STATUS_OK, rules);
 
+    rules = (Map) Utils.fromJSONString(permissions);
+    ((List)rules.get("permissions")).add( makeMap("name", "core-admin-edit", "role", "su"));
+    ((List)rules.get("permissions")).add( makeMap("name", "core-admin-read", "role", "user"));
+    ((Map)rules.get("user-role")).put("cio","su");
+    ((List)rules.get("permissions")).add( makeMap("name", "all", "role", "su"));
+    permissions = Utils.toJSONString(rules);
+
+    checkRules(makeMap("resource", "/admin/cores",
+        "userPrincipal", null,
+        "requestType", RequestType.ADMIN,
+        "collectionRequests", null,
+        "handler", new CoreAdminHandler(null),
+        "params", new MapSolrParams(singletonMap("action", "CREATE")))
+        , PROMPT_FOR_CREDENTIALS);
+
+    checkRules(makeMap("resource", "/admin/cores",
+        "userPrincipal", "joe",
+        "requestType", RequestType.ADMIN,
+        "collectionRequests", null,
+        "handler", new CoreAdminHandler(null),
+        "params", new MapSolrParams(singletonMap("action", "CREATE")))
+        , FORBIDDEN);
+
+  checkRules(makeMap("resource", "/admin/cores",
+        "userPrincipal", "joe",
+        "requestType", RequestType.ADMIN,
+        "collectionRequests", null,
+        "handler", new CoreAdminHandler(null),
+        "params", new MapSolrParams(singletonMap("action", "STATUS")))
+        , STATUS_OK);
+
+    checkRules(makeMap("resource", "/admin/cores",
+        "userPrincipal", "cio",
+        "requestType", RequestType.ADMIN,
+        "collectionRequests", null,
+        "handler", new CoreAdminHandler(null),
+        "params", new MapSolrParams(singletonMap("action", "CREATE")))
+        ,STATUS_OK );
+
   }
 
   public void testEditRules() throws IOException {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0233765c/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 0a9e749..f1728db 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
@@ -109,7 +109,7 @@ public abstract class CoreAdminParams
   public static final String NODE = "node";
 
   public enum CoreAdminAction {
-    STATUS,
+    STATUS(true),
     UNLOAD,
     RELOAD,
     CREATE,
@@ -118,18 +118,28 @@ public abstract class CoreAdminParams
     MERGEINDEXES,
     SPLIT,
     PREPRECOVERY,
-    REQUESTRECOVERY, 
+    REQUESTRECOVERY,
     REQUESTSYNCSHARD,
     DELETEALIAS,
     REQUESTBUFFERUPDATES,
     REQUESTAPPLYUPDATES,
     OVERSEEROP,
-    REQUESTSTATUS,
+    REQUESTSTATUS(true),
     REJOINLEADERELECTION,
     //internal API used by force shard leader election
     FORCEPREPAREFORLEADERSHIP,
     INVOKE;
 
+    public final boolean isRead;
+
+    CoreAdminAction(boolean isRead) {
+      this.isRead = isRead;
+    }
+
+    CoreAdminAction() {
+      this.isRead = false;
+    }
+
     public static CoreAdminAction get( String p ) {
       if (p != null) {
         try {


[24/50] [abbrv] lucene-solr:solr-5750: implement grow() for polygon queries too: easy speedup.

Posted by ds...@apache.org.
implement grow() for polygon queries too: easy speedup.


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

Branch: refs/heads/solr-5750
Commit: 0dc32e3d6b1a325f868980ae6fd8578af177ceb3
Parents: 45c48da
Author: Robert Muir <rm...@apache.org>
Authored: Sun Apr 24 18:47:50 2016 -0400
Committer: Robert Muir <rm...@apache.org>
Committed: Sun Apr 24 18:47:50 2016 -0400

----------------------------------------------------------------------
 .../org/apache/lucene/document/LatLonPointInPolygonQuery.java   | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0dc32e3d/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java
index 23a98d2..e68cb45 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java
@@ -121,6 +121,11 @@ final class LatLonPointInPolygonQuery extends Query {
         values.intersect(field, 
                          new IntersectVisitor() {
                            @Override
+                           public void grow(int count) {
+                             result.grow(count);
+                           }
+
+                           @Override
                            public void visit(int docID) {
                              result.add(docID);
                            }


[27/50] [abbrv] lucene-solr:solr-5750: fix stale javadocs

Posted by ds...@apache.org.
fix stale javadocs


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

Branch: refs/heads/solr-5750
Commit: fe795c9f7a5e936fe7ed6dd33e5d39105624683e
Parents: fd7b215
Author: Mike McCandless <mi...@apache.org>
Authored: Mon Apr 25 11:14:39 2016 -0400
Committer: Mike McCandless <mi...@apache.org>
Committed: Mon Apr 25 11:14:39 2016 -0400

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


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fe795c9f/lucene/core/src/java/org/apache/lucene/document/StringField.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/StringField.java b/lucene/core/src/java/org/apache/lucene/document/StringField.java
index b3c7fe0..7b968b6 100644
--- a/lucene/core/src/java/org/apache/lucene/document/StringField.java
+++ b/lucene/core/src/java/org/apache/lucene/document/StringField.java
@@ -23,8 +23,8 @@ import org.apache.lucene.util.BytesRef;
 /** A field that is indexed but not tokenized: the entire
  *  String value is indexed as a single token.  For example
  *  this might be used for a 'country' field or an 'id'
- *  field, or any field that you intend to use for sorting
- *  or access through the field cache. */
+ *  field.  If you also need to sort on this field, separately
+ *  add a {@link SortedDocValuesField} to your document. */
 
 public final class StringField extends Field {
 


[20/50] [abbrv] lucene-solr:solr-5750: LUCENE-7250: Handle holes properly for distance and relationship calculation.

Posted by ds...@apache.org.
LUCENE-7250: Handle holes properly for distance and relationship calculation.


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

Branch: refs/heads/solr-5750
Commit: 97e8f1aeadd29207b8fdc6284ec7b6e4c60cce11
Parents: 38ebd90
Author: Karl Wright <Da...@gmail.com>
Authored: Sat Apr 23 06:44:42 2016 -0400
Committer: Karl Wright <Da...@gmail.com>
Committed: Sat Apr 23 06:44:42 2016 -0400

----------------------------------------------------------------------
 .../org/apache/lucene/spatial3d/Geo3DPoint.java | 17 ++++------
 .../spatial3d/geom/GeoConcavePolygon.java       | 33 +++++++++++++++++---
 .../lucene/spatial3d/geom/GeoConvexPolygon.java | 33 +++++++++++++++++---
 .../spatial3d/geom/GeoPolygonFactory.java       |  6 ++--
 4 files changed, 68 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/97e8f1ae/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/Geo3DPoint.java
----------------------------------------------------------------------
diff --git a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/Geo3DPoint.java b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/Geo3DPoint.java
index 4b5ab92..0f2395c 100644
--- a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/Geo3DPoint.java
+++ b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/Geo3DPoint.java
@@ -142,7 +142,7 @@ public final class Geo3DPoint extends Field {
     }
     final GeoShape shape;
     if (polygons.length == 1) {
-      final GeoShape component = fromPolygon(polygons[0], false);
+      final GeoShape component = fromPolygon(polygons[0]);
       if (component == null) {
         // Polygon is degenerate
         shape = new GeoCompositePolygon();
@@ -152,7 +152,7 @@ public final class Geo3DPoint extends Field {
     } else {
       final GeoCompositePolygon poly = new GeoCompositePolygon();
       for (final Polygon p : polygons) {
-        final GeoPolygon component = fromPolygon(p, false);
+        final GeoPolygon component = fromPolygon(p);
         if (component != null) {
           poly.addShape(component);
         }
@@ -192,17 +192,16 @@ public final class Geo3DPoint extends Field {
     * Convert a Polygon object into a GeoPolygon.
     * This method uses
     * @param polygon is the Polygon object.
-    * @param reverseMe is true if the order of the points should be reversed.
     * @return the GeoPolygon.
     */
-  private static GeoPolygon fromPolygon(final Polygon polygon, final boolean reverseMe) {
+  private static GeoPolygon fromPolygon(final Polygon polygon) {
     // First, assemble the "holes".  The geo3d convention is to use the same polygon sense on the inner ring as the
     // outer ring, so we process these recursively with reverseMe flipped.
     final Polygon[] theHoles = polygon.getHoles();
     final List<GeoPolygon> holeList = new ArrayList<>(theHoles.length);
     for (final Polygon hole : theHoles) {
       //System.out.println("Hole: "+hole);
-      final GeoPolygon component = fromPolygon(hole, !reverseMe);
+      final GeoPolygon component = fromPolygon(hole);
       if (component != null) {
         holeList.add(component);
       }
@@ -216,12 +215,8 @@ public final class Geo3DPoint extends Field {
     final List<GeoPoint> points = new ArrayList<>(polyLats.length-1);
     // We skip the last point anyway because the API requires it to be repeated, and geo3d doesn't repeat it.
     for (int i = 0; i < polyLats.length - 1; i++) {
-      if (reverseMe) {
-        points.add(new GeoPoint(PlanetModel.WGS84, fromDegrees(polyLats[i]), fromDegrees(polyLons[i])));
-      } else {
-        final int index = polyLats.length - 2 - i;
-        points.add(new GeoPoint(PlanetModel.WGS84, fromDegrees(polyLats[index]), fromDegrees(polyLons[index])));
-      }
+      final int index = polyLats.length - 2 - i;
+      points.add(new GeoPoint(PlanetModel.WGS84, fromDegrees(polyLats[index]), fromDegrees(polyLons[index])));
     }
     //System.err.println(" building polygon with "+points.size()+" points...");
     final GeoPolygon rval = GeoPolygonFactory.makeGeoPolygon(PlanetModel.WGS84, points, holeList);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/97e8f1ae/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoConcavePolygon.java
----------------------------------------------------------------------
diff --git a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoConcavePolygon.java b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoConcavePolygon.java
index 8eaea1a..c18d40f 100644
--- a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoConcavePolygon.java
+++ b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoConcavePolygon.java
@@ -224,8 +224,25 @@ class GeoConcavePolygon extends GeoBasePolygon {
       eitherBounds.put(edge, new EitherBound(invertedEdges[legalIndex(bound1Index)], invertedEdges[legalIndex(bound2Index)]));
     }
 
-    // Pick an edge point arbitrarily
-    edgePoints = new GeoPoint[]{points.get(0)};
+    // Pick an edge point arbitrarily from the outer polygon.  Glom this together with all edge points from
+    // inner polygons.
+    int edgePointCount = 1;
+    if (holes != null) {
+      for (final GeoPolygon hole : holes) {
+        edgePointCount += hole.getEdgePoints().length;
+      }
+    }
+    edgePoints = new GeoPoint[edgePointCount];
+    edgePointCount = 0;
+    edgePoints[edgePointCount++] = points.get(0);
+    if (holes != null) {
+      for (final GeoPolygon hole : holes) {
+        final GeoPoint[] holeEdgePoints = hole.getEdgePoints();
+        for (final GeoPoint p : holeEdgePoints) {
+          edgePoints[edgePointCount++] = p;
+        }
+      }
+    }
 
     if (isWithinHoles(points.get(0))) {
       throw new IllegalArgumentException("Polygon edge intersects a polygon hole; not allowed");
@@ -240,7 +257,7 @@ class GeoConcavePolygon extends GeoBasePolygon {
   protected boolean isWithinHoles(final GeoPoint point) {
     if (holes != null) {
       for (final GeoPolygon hole : holes) {
-        if (hole.isWithin(point)) {
+        if (!hole.isWithin(point)) {
           return true;
         }
       }
@@ -268,7 +285,7 @@ class GeoConcavePolygon extends GeoBasePolygon {
     }
     if (holes != null) {
       for (final GeoPolygon polygon : holes) {
-        if (polygon.isWithin(x, y, z)) {
+        if (!polygon.isWithin(x, y, z)) {
           return false;
         }
       }
@@ -405,6 +422,14 @@ class GeoConcavePolygon extends GeoBasePolygon {
         minimumDistance = newDist;
       }
     }
+    if (holes != null) {
+      for (final GeoPolygon hole : holes) {
+        double holeDistance = hole.computeOutsideDistance(distanceStyle, x, y, z);
+        if (holeDistance != 0.0 && holeDistance < minimumDistance) {
+          minimumDistance = holeDistance;
+        }
+      }
+    }
     return minimumDistance;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/97e8f1ae/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoConvexPolygon.java
----------------------------------------------------------------------
diff --git a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoConvexPolygon.java b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoConvexPolygon.java
index 17a2120..6f71d18 100755
--- a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoConvexPolygon.java
+++ b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoConvexPolygon.java
@@ -219,8 +219,25 @@ class GeoConvexPolygon extends GeoBasePolygon {
       eitherBounds.put(edge, new EitherBound(edges[legalIndex(bound1Index)], edges[legalIndex(bound2Index)]));
     }
     
-    // Pick an edge point arbitrarily
-    edgePoints = new GeoPoint[]{points.get(0)};
+    // Pick an edge point arbitrarily from the outer polygon.  Glom this together with all edge points from
+    // inner polygons.
+    int edgePointCount = 1;
+    if (holes != null) {
+      for (final GeoPolygon hole : holes) {
+        edgePointCount += hole.getEdgePoints().length;
+      }
+    }
+    edgePoints = new GeoPoint[edgePointCount];
+    edgePointCount = 0;
+    edgePoints[edgePointCount++] = points.get(0);
+    if (holes != null) {
+      for (final GeoPolygon hole : holes) {
+        final GeoPoint[] holeEdgePoints = hole.getEdgePoints();
+        for (final GeoPoint p : holeEdgePoints) {
+          edgePoints[edgePointCount++] = p;
+        }
+      }
+    }
     
     if (isWithinHoles(points.get(0))) {
       throw new IllegalArgumentException("Polygon edge intersects a polygon hole; not allowed");
@@ -235,7 +252,7 @@ class GeoConvexPolygon extends GeoBasePolygon {
   protected boolean isWithinHoles(final GeoPoint point) {
     if (holes != null) {
       for (final GeoPolygon hole : holes) {
-        if (hole.isWithin(point)) {
+        if (!hole.isWithin(point)) {
           return true;
         }
       }
@@ -263,7 +280,7 @@ class GeoConvexPolygon extends GeoBasePolygon {
     }
     if (holes != null) {
       for (final GeoPolygon polygon : holes) {
-        if (polygon.isWithin(x, y, z)) {
+        if (!polygon.isWithin(x, y, z)) {
           return false;
         }
       }
@@ -393,6 +410,14 @@ class GeoConvexPolygon extends GeoBasePolygon {
         minimumDistance = newDist;
       }
     }
+    if (holes != null) {
+      for (final GeoPolygon hole : holes) {
+        double holeDistance = hole.computeOutsideDistance(distanceStyle, x, y, z);
+        if (holeDistance != 0.0 && holeDistance < minimumDistance) {
+          minimumDistance = holeDistance;
+        }
+      }
+    }
     return minimumDistance;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/97e8f1ae/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoPolygonFactory.java
----------------------------------------------------------------------
diff --git a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoPolygonFactory.java b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoPolygonFactory.java
index 99fc7c9..609c864 100755
--- a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoPolygonFactory.java
+++ b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoPolygonFactory.java
@@ -56,7 +56,8 @@ public class GeoPolygonFactory {
    * @param pointList is a list of the GeoPoints to build an arbitrary polygon out of.  If points go
    *  clockwise from a given pole, then that pole should be within the polygon.  If points go
    *  counter-clockwise, then that pole should be outside the polygon.
-   * @param holes is a list of polygons representing "holes" in the outside polygon.  Null == none.
+   * @param holes is a list of polygons representing "holes" in the outside polygon.  Holes describe the area outside
+   *  each hole as being "in set".  Null == none.
    * @return a GeoPolygon corresponding to what was specified, or null if a valid polygon cannot be generated
    *  from this input.
    */
@@ -73,7 +74,8 @@ public class GeoPolygonFactory {
    * @param pointList is a list of the GeoPoints to build an arbitrary polygon out of.  If points go
    *  clockwise from a given pole, then that pole should be within the polygon.  If points go
    *  counter-clockwise, then that pole should be outside the polygon.
-   * @param holes is a list of polygons representing "holes" in the outside polygon.  Null == none.
+   * @param holes is a list of polygons representing "holes" in the outside polygon.  Holes describe the area outside
+   *  each hole as being "in set".  Null == none.
    * @param leniencyValue is the maximum distance (in units) that a point can be from the plane and still be considered as
    *  belonging to the plane.  Any value greater than zero may cause some of the provided points that are in fact outside
    *  the strict definition of co-planarity, but are within this distance, to be discarded for the purposes of creating a


[45/50] [abbrv] lucene-solr:solr-5750: SOLR-5750: refactor: duplicate applicable POJO state of CollectionAdminRequest.Create to Restore

Posted by ds...@apache.org.
SOLR-5750: refactor: duplicate applicable POJO state of CollectionAdminRequest.Create to Restore


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

Branch: refs/heads/solr-5750
Commit: 66b56e61c9b67f964e5f17fcf555514aada7ebd9
Parents: 17c7a22
Author: David Smiley <ds...@apache.org>
Authored: Wed Apr 27 10:52:10 2016 -0400
Committer: David Smiley <ds...@apache.org>
Committed: Wed Apr 27 10:52:10 2016 -0400

----------------------------------------------------------------------
 .../solr/cloud/TestCloudBackupRestore.java      |  6 +-
 .../solrj/request/CollectionAdminRequest.java   | 71 ++++++++++++--------
 2 files changed, 46 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/66b56e61/solr/core/src/test/org/apache/solr/cloud/TestCloudBackupRestore.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestCloudBackupRestore.java b/solr/core/src/test/org/apache/solr/cloud/TestCloudBackupRestore.java
index 222c853..9354040 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestCloudBackupRestore.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestCloudBackupRestore.java
@@ -164,13 +164,13 @@ public class TestCloudBackupRestore extends SolrCloudTestCase {
               .setLocation(location);
       if (origShardToDocCount.size() > cluster.getJettySolrRunners().size()) {
         // may need to increase maxShardsPerNode (e.g. if it was shard split, then now we need more)
-        restore.getCreateOptions().setMaxShardsPerNode(origShardToDocCount.size());
+        restore.setMaxShardsPerNode(origShardToDocCount.size());
       }
       Properties props = new Properties();
       props.setProperty("customKey", "customVal");
-      restore.getCreateOptions().setProperties(props);
+      restore.setProperties(props);
       if (sameConfig==false) {
-        restore.getCreateOptions().setConfigName("customConfigName");//nocommit ugh, this is deprecated
+        restore.setConfigName("customConfigName");
       }
       if (random().nextBoolean()) {
         assertEquals(0, restore.process(client).getStatus());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/66b56e61/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
index 5fe2e1c..4af5c00 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
@@ -567,8 +567,8 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
     @Override
     public SolrParams getParams() {
       ModifiableSolrParams params = (ModifiableSolrParams) super.getParams();
-      params.set(CoreAdminParams.COLLECTION, collection);
-      params.set(CoreAdminParams.NAME, name);
+      params.set(CoreAdminParams.COLLECTION, collection); // nocommit reconsider to use NAME (set via super.getParams)
+      params.set(CoreAdminParams.NAME, name); // nocommit reconsider to use backupName?
       params.set("location", location); //note: optional
       return params;
     }
@@ -583,7 +583,13 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
   public static class Restore extends AsyncCollectionSpecificAdminRequest {
     protected final String backupName;
     protected String location;
-    protected Create createOptions;//lazy created
+
+    // in common with collection creation:
+    protected String configName;
+    protected Integer maxShardsPerNode;
+    protected Integer replicationFactor;
+    protected Boolean autoAddReplicas;
+    protected Properties properties;
 
     public Restore(String collection, String backupName) {
       super(CollectionAction.RESTORE, collection);
@@ -611,36 +617,45 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
       return this;
     }
 
-    /** The returned create command is used as a POJO to pass additional parameters -- restoring a collection involves
-     * creating a collection.  However, note some options aren't supported like numShards and configuring the router.
-     */
-    // note: it was either this (hack?) or we extend Create which would be weird/hack, or we verbosely duplicate lots of
-    //   POJO methods, or we perhaps modify the base class to have an addParam() but we loose typed/documented options.
-    //   nocommit but unfortunately, setConfigName & setReplicationFactor are deprecated. Now what?
-    public Create getCreateOptions() {
-      if (createOptions == null) {
-        createOptions = new Create(collection, null, -1, -1);
-      }
-      return createOptions;
+    // Collection creation params in common:
+    public Restore setConfigName(String config) { this.configName = config; return this; }
+    public String getConfigName()  { return configName; }
+
+    public Integer getMaxShardsPerNode() { return maxShardsPerNode; }
+    public Restore setMaxShardsPerNode(int maxShardsPerNode) { this.maxShardsPerNode = maxShardsPerNode; return this; }
+
+    public Integer getReplicationFactor() { return replicationFactor; }
+    public Restore setReplicationFactor(Integer repl) { this.replicationFactor = repl; return this; }
+
+    public Boolean getAutoAddReplicas() { return autoAddReplicas; }
+    public Restore setAutoAddReplicas(boolean autoAddReplicas) { this.autoAddReplicas = autoAddReplicas; return this; }
+
+    public Properties getProperties() {
+      return properties;
     }
+    public Restore setProperties(Properties properties) { this.properties = properties; return this;}
+
+    // TODO support createNodeSet, rule, snitch
 
     @Override
     public SolrParams getParams() {
-      ModifiableSolrParams params;
-      if (createOptions != null) {
-        params = (ModifiableSolrParams) createOptions.getParams();
-        // remove these two settings that create() made us set (unless customized)
-        if ("-1".equals(params.get("replicationFactor"))) {
-          params.remove("replicationFactor");
-        }
-        params.remove("numShards"); // not customizable
-        params.add(super.getParams());//override action, and we override some below too...
-      } else {
-        params = (ModifiableSolrParams) super.getParams();
-      }
-      params.set(CoreAdminParams.COLLECTION, collection);
-      params.set(CoreAdminParams.NAME, backupName);
+      ModifiableSolrParams params = (ModifiableSolrParams) super.getParams();
+      params.set(CoreAdminParams.COLLECTION, collection); // nocommit reconsider to use NAME (set via super.getParams)
+      params.set(CoreAdminParams.NAME, backupName); // nocommit reconsider to use backupName?
       params.set("location", location); //note: optional
+      params.set("collection.configName", configName); //note: optional
+      if (maxShardsPerNode != null) {
+        params.set( "maxShardsPerNode", maxShardsPerNode);
+      }
+      if (replicationFactor != null) {
+        params.set("replicationFactor", replicationFactor);
+      }
+      if (autoAddReplicas != null) {
+        params.set(ZkStateReader.AUTO_ADD_REPLICAS, autoAddReplicas);
+      }
+      if (properties != null) {
+        addProperties(params, properties);
+      }
       return params;
     }
 


[05/50] [abbrv] lucene-solr:solr-5750: LUCENE-7235: Avoid taking the lock in LRUQueryCache when not necessary.

Posted by ds...@apache.org.
LUCENE-7235: Avoid taking the lock in LRUQueryCache when not necessary.


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

Branch: refs/heads/solr-5750
Commit: 81446cf34531d46f224beaf6c2bc70bdf53ee585
Parents: 2ff8bb0
Author: Adrien Grand <jp...@gmail.com>
Authored: Thu Apr 21 15:20:03 2016 +0200
Committer: Adrien Grand <jp...@gmail.com>
Committed: Thu Apr 21 15:31:13 2016 +0200

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  3 +
 .../org/apache/lucene/search/LRUQueryCache.java | 69 +++++++++++++++++--
 .../lucene/search/QueryCachingPolicy.java       | 51 --------------
 .../search/UsageTrackingQueryCachingPolicy.java | 22 +-----
 .../apache/lucene/search/TestLRUQueryCache.java | 71 +++++++++++++++-----
 .../lucene/search/TestQueryCachingPolicy.java   | 56 ---------------
 .../org/apache/lucene/util/LuceneTestCase.java  |  2 +-
 7 files changed, 125 insertions(+), 149 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81446cf3/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index b0abbc1..7c30ef4 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -70,6 +70,9 @@ Optimizations
 * LUCENE-7211: Reduce memory & GC for spatial RPT Intersects when the number of
   matching docs is small. (Jeff Wartes, David Smiley)
 
+* LUCENE-7235: LRUQueryCache should not take a lock for segments that it will
+  not cache on anyway. (Adrien Grand)
+
 Bug Fixes
 
 * LUCENE-7127: Fix corner case bugs in GeoPointDistanceQuery. (Robert Muir)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81446cf3/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java b/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
index 6c065ae..f4cf8dc 100644
--- a/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
+++ b/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
@@ -29,11 +29,14 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Predicate;
 
 import org.apache.lucene.index.LeafReader.CoreClosedListener;
+import org.apache.lucene.index.IndexReaderContext;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.ReaderUtil;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.index.TieredMergePolicy;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.Accountables;
 import org.apache.lucene.util.RamUsageEstimator;
@@ -100,6 +103,7 @@ public class LRUQueryCache implements QueryCache, Accountable {
 
   private final int maxSize;
   private final long maxRamBytesUsed;
+  private final Predicate<LeafReaderContext> leavesToCache;
   // maps queries that are contained in the cache to a singleton so that this
   // cache does not store several copies of the same query
   private final Map<Query, Query> uniqueQueries;
@@ -118,12 +122,15 @@ public class LRUQueryCache implements QueryCache, Accountable {
   private volatile long cacheSize;
 
   /**
-   * Create a new instance that will cache at most <code>maxSize</code> queries
-   * with at most <code>maxRamBytesUsed</code> bytes of memory.
+   * Expert: Create a new instance that will cache at most <code>maxSize</code>
+   * queries with at most <code>maxRamBytesUsed</code> bytes of memory, only on
+   * leaves that satisfy {@code leavesToCache};
    */
-  public LRUQueryCache(int maxSize, long maxRamBytesUsed) {
+  public LRUQueryCache(int maxSize, long maxRamBytesUsed,
+      Predicate<LeafReaderContext> leavesToCache) {
     this.maxSize = maxSize;
     this.maxRamBytesUsed = maxRamBytesUsed;
+    this.leavesToCache = leavesToCache;
     uniqueQueries = new LinkedHashMap<>(16, 0.75f, true);
     mostRecentlyUsedQueries = uniqueQueries.keySet();
     cache = new IdentityHashMap<>();
@@ -131,6 +138,43 @@ public class LRUQueryCache implements QueryCache, Accountable {
   }
 
   /**
+   * Create a new instance that will cache at most <code>maxSize</code> queries
+   * with at most <code>maxRamBytesUsed</code> bytes of memory. Queries will
+   * only be cached on leaves that have more than 10k documents and have more
+   * than 3% of the total number of documents in the index.
+   * This should guarantee that all leaves from the upper
+   * {@link TieredMergePolicy tier} will be cached while ensuring that at most
+   * <tt>33</tt> leaves can make it to the cache (very likely less than 10 in
+   * practice), which is useful for this implementation since some operations
+   * perform in linear time with the number of cached leaves.
+   */
+  public LRUQueryCache(int maxSize, long maxRamBytesUsed) {
+    this(maxSize, maxRamBytesUsed, new MinSegmentSizePredicate(10000, .03f));
+  }
+
+  // pkg-private for testing
+  static class MinSegmentSizePredicate implements Predicate<LeafReaderContext> {
+    private final int minSize;
+    private final float minSizeRatio;
+
+    MinSegmentSizePredicate(int minSize, float minSizeRatio) {
+      this.minSize = minSize;
+      this.minSizeRatio = minSizeRatio;
+    }
+
+    @Override
+    public boolean test(LeafReaderContext context) {
+      final int maxDoc = context.reader().maxDoc();
+      if (maxDoc < minSize) {
+        return false;
+      }
+      final IndexReaderContext topLevelContext = ReaderUtil.getTopLevelContext(context);
+      final float sizeRatio = (float) context.reader().maxDoc() / topLevelContext.reader().maxDoc();
+      return sizeRatio >= minSizeRatio;
+    }
+  }
+
+  /**
    * Expert: callback when there is a cache hit on a given query.
    * Implementing this method is typically useful in order to compute more
    * fine-grained statistics about the query cache.
@@ -598,9 +642,10 @@ public class LRUQueryCache implements QueryCache, Accountable {
       }
     }
 
+    /** Check whether this segment is eligible for caching, regardless of the query. */
     private boolean shouldCache(LeafReaderContext context) throws IOException {
       return cacheEntryHasReasonableWorstCaseSize(ReaderUtil.getTopLevelContext(context).reader().maxDoc())
-          && policy.shouldCache(in.getQuery(), context);
+          && leavesToCache.test(context);
     }
 
     @Override
@@ -608,9 +653,15 @@ public class LRUQueryCache implements QueryCache, Accountable {
       if (used.compareAndSet(false, true)) {
         policy.onUse(getQuery());
       }
+      // Short-circuit: Check whether this segment is eligible for caching
+      // before we take a lock because of #get
+      if (shouldCache(context) == false) {
+        return in.scorer(context);
+      }
+
       DocIdSet docIdSet = get(in.getQuery(), context);
       if (docIdSet == null) {
-        if (shouldCache(context)) {
+        if (policy.shouldCache(in.getQuery(), context)) {
           docIdSet = cache(context);
           putIfAbsent(in.getQuery(), context, docIdSet);
         } else {
@@ -635,9 +686,15 @@ public class LRUQueryCache implements QueryCache, Accountable {
       if (used.compareAndSet(false, true)) {
         policy.onUse(getQuery());
       }
+      // Short-circuit: Check whether this segment is eligible for caching
+      // before we take a lock because of #get
+      if (shouldCache(context) == false) {
+        return in.bulkScorer(context);
+      }
+
       DocIdSet docIdSet = get(in.getQuery(), context);
       if (docIdSet == null) {
-        if (shouldCache(context)) {
+        if (policy.shouldCache(in.getQuery(), context)) {
           docIdSet = cache(context);
           putIfAbsent(in.getQuery(), context, docIdSet);
         } else {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81446cf3/lucene/core/src/java/org/apache/lucene/search/QueryCachingPolicy.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/QueryCachingPolicy.java b/lucene/core/src/java/org/apache/lucene/search/QueryCachingPolicy.java
index d62779e..c0f6aa8 100644
--- a/lucene/core/src/java/org/apache/lucene/search/QueryCachingPolicy.java
+++ b/lucene/core/src/java/org/apache/lucene/search/QueryCachingPolicy.java
@@ -19,10 +19,7 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
-import org.apache.lucene.index.IndexReaderContext;
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.ReaderUtil;
-import org.apache.lucene.index.TieredMergePolicy;
 
 /**
  * A policy defining which filters should be cached.
@@ -49,54 +46,6 @@ public interface QueryCachingPolicy {
 
   };
 
-  /** A simple policy that only caches on the largest segments of an index.
-   *  The reasoning is that these segments likely account for most of the
-   *  execution time of queries and are also more likely to stay around longer
-   *  than small segments, which makes them more interesting for caching.
-   */
-  public static class CacheOnLargeSegments implements QueryCachingPolicy {
-
-    /** {@link CacheOnLargeSegments} instance that only caches on segments that
-     *  account for more than 3% of the total index size. This should guarantee
-     *  that all segments from the upper {@link TieredMergePolicy tier} will be
-     *  cached while ensuring that at most <tt>33</tt> segments can make it to
-     *  the cache (given that some implementations such as {@link LRUQueryCache}
-     *  perform better when the number of cached segments is low). */
-    public static final CacheOnLargeSegments DEFAULT = new CacheOnLargeSegments(10000, 0.03f);
-
-    private final int minIndexSize;
-    private final float minSizeRatio;
-
-    /**
-     * Create a {@link CacheOnLargeSegments} instance that only caches on a
-     * given segment if the total number of documents in the index is greater
-     * than {@code minIndexSize} and the number of documents in the segment
-     * divided by the total number of documents in the index is greater than
-     * or equal to {@code minSizeRatio}.
-     */
-    public CacheOnLargeSegments(int minIndexSize, float minSizeRatio) {
-      if (minSizeRatio <= 0 || minSizeRatio >= 1) {
-        throw new IllegalArgumentException("minSizeRatio must be in ]0, 1[, got " + minSizeRatio);
-      }
-      this.minIndexSize = minIndexSize;
-      this.minSizeRatio = minSizeRatio;
-    }
-
-    @Override
-    public void onUse(Query query) {}
-
-    @Override
-    public boolean shouldCache(Query query, LeafReaderContext context) throws IOException {
-      final IndexReaderContext topLevelContext = ReaderUtil.getTopLevelContext(context);
-      if (topLevelContext.reader().maxDoc() < minIndexSize) {
-        return false;
-      }
-      final float sizeRatio = (float) context.reader().maxDoc() / topLevelContext.reader().maxDoc();
-      return sizeRatio >= minSizeRatio;
-    }
-
-  };
-
   /** Callback that is called every time that a cached filter is used.
    *  This is typically useful if the policy wants to track usage statistics
    *  in order to make decisions. */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81446cf3/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java b/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java
index 417de58..4eb9289 100644
--- a/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java
+++ b/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java
@@ -66,33 +66,20 @@ public final class UsageTrackingQueryCachingPolicy implements QueryCachingPolicy
     return query instanceof TermQuery;
   }
 
-  private final QueryCachingPolicy.CacheOnLargeSegments segmentPolicy;
   private final FrequencyTrackingRingBuffer recentlyUsedFilters;
 
   /**
    * Create a new instance.
    *
-   * @param minIndexSize              the minimum size of the top-level index
-   * @param minSizeRatio              the minimum size ratio for segments to be cached, see {@link QueryCachingPolicy.CacheOnLargeSegments}
    * @param historySize               the number of recently used filters to track
    */
-  public UsageTrackingQueryCachingPolicy(
-      int minIndexSize,
-      float minSizeRatio,
-      int historySize) {
-    this(new QueryCachingPolicy.CacheOnLargeSegments(minIndexSize, minSizeRatio), historySize);
+  public UsageTrackingQueryCachingPolicy(int historySize) {
+    this.recentlyUsedFilters = new FrequencyTrackingRingBuffer(historySize, SENTINEL);
   }
 
   /** Create a new instance with an history size of 256. */
   public UsageTrackingQueryCachingPolicy() {
-    this(QueryCachingPolicy.CacheOnLargeSegments.DEFAULT, 256);
-  }
-
-  private UsageTrackingQueryCachingPolicy(
-      QueryCachingPolicy.CacheOnLargeSegments segmentPolicy,
-      int historySize) {
-    this.segmentPolicy = segmentPolicy;
-    this.recentlyUsedFilters = new FrequencyTrackingRingBuffer(historySize, SENTINEL);
+    this(256);
   }
 
   /**
@@ -160,9 +147,6 @@ public final class UsageTrackingQueryCachingPolicy implements QueryCachingPolicy
         return false;
       }
     }
-    if (segmentPolicy.shouldCache(query, context) == false) {
-      return false;
-    }
     final int frequency = frequency(query);
     final int minFrequency = minFrequencyToCache(query);
     return frequency >= minFrequency;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81446cf3/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
index 10131ea..2552b04 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
@@ -81,7 +81,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
   }
 
   public void testConcurrency() throws Throwable {
-    final LRUQueryCache queryCache = new LRUQueryCache(1 + random().nextInt(20), 1 + random().nextInt(10000));
+    final LRUQueryCache queryCache = new LRUQueryCache(1 + random().nextInt(20), 1 + random().nextInt(10000), context -> random().nextBoolean());
     Directory dir = newDirectory();
     final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
     final SearcherFactory searcherFactory = new SearcherFactory() {
@@ -184,7 +184,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
     w.addDocument(doc);
     final DirectoryReader reader = w.getReader();
     final IndexSearcher searcher = newSearcher(reader);
-    final LRUQueryCache queryCache = new LRUQueryCache(2, 100000);
+    final LRUQueryCache queryCache = new LRUQueryCache(2, 100000, context -> true);
 
     final Query blue = new TermQuery(new Term("color", "blue"));
     final Query red = new TermQuery(new Term("color", "red"));
@@ -245,7 +245,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
     // different instance yet equal
     final Query query2 = new TermQuery(new Term("color", "blue"));
 
-    final LRUQueryCache queryCache = new LRUQueryCache(Integer.MAX_VALUE, Long.MAX_VALUE);
+    final LRUQueryCache queryCache = new LRUQueryCache(Integer.MAX_VALUE, Long.MAX_VALUE, context -> true);
     searcher.setQueryCache(queryCache);
     searcher.setQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE);
 
@@ -265,7 +265,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
   // This test makes sure that by making the same assumptions as LRUQueryCache, RAMUsageTester
   // computes the same memory usage.
   public void testRamBytesUsedAgreesWithRamUsageTester() throws IOException {
-    final LRUQueryCache queryCache = new LRUQueryCache(1 + random().nextInt(5), 1 + random().nextInt(10000));
+    final LRUQueryCache queryCache = new LRUQueryCache(1 + random().nextInt(5), 1 + random().nextInt(10000), context -> random().nextBoolean());
     // an accumulator that only sums up memory usage of referenced filters and doc id sets
     final RamUsageTester.Accumulator acc = new RamUsageTester.Accumulator() {
       @Override
@@ -381,7 +381,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
   // by the cache itself, not cache entries, and we want to make sure that
   // memory usage is not grossly underestimated.
   public void testRamBytesUsedConstantEntryOverhead() throws IOException {
-    final LRUQueryCache queryCache = new LRUQueryCache(1000000, 10000000);
+    final LRUQueryCache queryCache = new LRUQueryCache(1000000, 10000000, context -> true);
 
     final RamUsageTester.Accumulator acc = new RamUsageTester.Accumulator() {
       @Override
@@ -430,7 +430,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
   }
 
   public void testOnUse() throws IOException {
-    final LRUQueryCache queryCache = new LRUQueryCache(1 + random().nextInt(5), 1 + random().nextInt(1000));
+    final LRUQueryCache queryCache = new LRUQueryCache(1 + random().nextInt(5), 1 + random().nextInt(1000), context -> random().nextBoolean());
 
     Directory dir = newDirectory();
     final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
@@ -490,7 +490,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
   }
 
   public void testStats() throws IOException {
-    final LRUQueryCache queryCache = new LRUQueryCache(1, 10000000);
+    final LRUQueryCache queryCache = new LRUQueryCache(1, 10000000, context -> true);
 
     Directory dir = newDirectory();
     final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
@@ -621,7 +621,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
     final AtomicLong ramBytesUsage = new AtomicLong();
     final AtomicLong cacheSize = new AtomicLong();
 
-    final LRUQueryCache queryCache = new LRUQueryCache(2, 10000000) {
+    final LRUQueryCache queryCache = new LRUQueryCache(2, 10000000, context -> true) {
       @Override
       protected void onHit(Object readerCoreKey, Query query) {
         super.onHit(readerCoreKey, query);
@@ -748,7 +748,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
     final BooleanQuery.Builder query = new BooleanQuery.Builder();
     query.add(new BoostQuery(expectedCacheKey, 42f), Occur.MUST);
 
-    final LRUQueryCache queryCache = new LRUQueryCache(1000000, 10000000);
+    final LRUQueryCache queryCache = new LRUQueryCache(1000000, 10000000, context -> random().nextBoolean());
     Directory dir = newDirectory();
     final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
     Document doc = new Document();
@@ -793,7 +793,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
     final IndexSearcher searcher = newSearcher(reader);
     w.close();
 
-    final LRUQueryCache queryCache = new LRUQueryCache(1000000, 10000000);
+    final LRUQueryCache queryCache = new LRUQueryCache(1000000, 10000000, context -> true);
     searcher.setQueryCache(queryCache);
     searcher.setQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE);
 
@@ -897,7 +897,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
       iters = atLeast(2000);
     }
 
-    final LRUQueryCache queryCache = new LRUQueryCache(maxSize, maxRamBytesUsed);
+    final LRUQueryCache queryCache = new LRUQueryCache(maxSize, maxRamBytesUsed, context -> random().nextBoolean());
     IndexSearcher uncachedSearcher = null;
     IndexSearcher cachedSearcher = null;
 
@@ -962,7 +962,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
     IndexReader reader = w.getReader();
 
     // size of 1 so that 2nd query evicts from the cache
-    final LRUQueryCache queryCache = new LRUQueryCache(1, 10000);
+    final LRUQueryCache queryCache = new LRUQueryCache(1, 10000, context -> true);
     final IndexSearcher searcher = newSearcher(reader);
     searcher.setQueryCache(queryCache);
     searcher.setQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE);
@@ -996,7 +996,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
     IndexReader reader = w.getReader();
 
     // size of 1 byte
-    final LRUQueryCache queryCache = new LRUQueryCache(1, 1);
+    final LRUQueryCache queryCache = new LRUQueryCache(1, 1, context -> random().nextBoolean());
     final IndexSearcher searcher = newSearcher(reader);
     searcher.setQueryCache(queryCache);
     searcher.setQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE);
@@ -1035,7 +1035,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
       try (final IndexReader indexReader = DirectoryReader.open(directory)) {
         final FrequencyCountingPolicy policy = new FrequencyCountingPolicy();
         final IndexSearcher indexSearcher = new IndexSearcher(indexReader);
-        indexSearcher.setQueryCache(new LRUQueryCache(100, 10240));
+        indexSearcher.setQueryCache(new LRUQueryCache(100, 10240, context -> random().nextBoolean()));
         indexSearcher.setQueryCachingPolicy(policy);
         final Query foo = new TermQuery(new Term("f", "foo"));
         final Query bar = new TermQuery(new Term("f", "bar"));
@@ -1141,7 +1141,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
     LeafReaderContext leaf = searcher.getIndexReader().leaves().get(0);
     AtomicBoolean scorerCalled = new AtomicBoolean();
     AtomicBoolean bulkScorerCalled = new AtomicBoolean();
-    LRUQueryCache cache = new LRUQueryCache(1, Long.MAX_VALUE);
+    LRUQueryCache cache = new LRUQueryCache(1, Long.MAX_VALUE, context -> true);
 
     // test that the bulk scorer is propagated when a scorer should not be cached
     Weight weight = searcher.createNormalizedWeight(new MatchAllDocsQuery(), false);
@@ -1172,7 +1172,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
     w.addDocument(new Document());
     final DirectoryReader reader = w.getReader();
     final IndexSearcher searcher = newSearcher(reader);
-    final LRUQueryCache queryCache = new LRUQueryCache(2, 100000) {
+    final LRUQueryCache queryCache = new LRUQueryCache(2, 100000, context -> true) {
       @Override
       protected void onDocIdSetEviction(Object readerCoreKey, int numEntries, long sumRamBytesUsed) {
         super.onDocIdSetEviction(readerCoreKey, numEntries, sumRamBytesUsed);
@@ -1192,4 +1192,43 @@ public class TestLRUQueryCache extends LuceneTestCase {
     w.close();
     dir.close();
   }
+
+  public void testMinSegmentSizePredicate() throws IOException {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = newIndexWriterConfig().setMergePolicy(NoMergePolicy.INSTANCE);
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
+    w.addDocument(new Document());
+    DirectoryReader reader = w.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+    searcher.setQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE);
+
+    LRUQueryCache cache = new LRUQueryCache(2, 10000, new LRUQueryCache.MinSegmentSizePredicate(2, 0f));
+    searcher.setQueryCache(cache);
+    searcher.count(new DummyQuery());
+    assertEquals(0, cache.getCacheCount());
+
+    cache = new LRUQueryCache(2, 10000, new LRUQueryCache.MinSegmentSizePredicate(1, 0f));
+    searcher.setQueryCache(cache);
+    searcher.count(new DummyQuery());
+    assertEquals(1, cache.getCacheCount());
+
+    cache = new LRUQueryCache(2, 10000, new LRUQueryCache.MinSegmentSizePredicate(0, .6f));
+    searcher.setQueryCache(cache);
+    searcher.count(new DummyQuery());
+    assertEquals(1, cache.getCacheCount());
+
+    w.addDocument(new Document());
+    reader.close();
+    reader = w.getReader();
+    searcher = newSearcher(reader);
+    searcher.setQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE);
+    cache = new LRUQueryCache(2, 10000, new LRUQueryCache.MinSegmentSizePredicate(0, .6f));
+    searcher.setQueryCache(cache);
+    searcher.count(new DummyQuery());
+    assertEquals(0, cache.getCacheCount());
+
+    reader.close();
+    w.close();
+    dir.close();
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81446cf3/lucene/core/src/test/org/apache/lucene/search/TestQueryCachingPolicy.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestQueryCachingPolicy.java b/lucene/core/src/test/org/apache/lucene/search/TestQueryCachingPolicy.java
deleted file mode 100644
index 29ef6ba..0000000
--- a/lucene/core/src/test/org/apache/lucene/search/TestQueryCachingPolicy.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.search;
-
-
-import java.io.IOException;
-
-import org.apache.lucene.document.Document;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.LuceneTestCase;
-
-public class TestQueryCachingPolicy extends LuceneTestCase {
-
-  public void testLargeSegmentDetection() throws IOException {
-    Directory dir = newDirectory();
-    RandomIndexWriter w = new RandomIndexWriter(random(), dir);
-    final int numDocs = atLeast(100);
-    for (int i = 0; i < numDocs; ++i) {
-      w.addDocument(new Document());
-    }
-    final IndexReader reader = w.getReader();
-    for (float minSizeRatio : new float[] {Float.MIN_VALUE, 0.01f, 0.1f, 0.9f}) {
-      final QueryCachingPolicy policy = new QueryCachingPolicy.CacheOnLargeSegments(0, minSizeRatio);
-      for (LeafReaderContext ctx : reader.leaves()) {
-        final Query query = new TermQuery(new Term("field", "value"));
-        final boolean shouldCache = policy.shouldCache(query, ctx);
-        final float sizeRatio = (float) ctx.reader().maxDoc() / reader.maxDoc();
-        assertEquals(sizeRatio >= minSizeRatio, shouldCache);
-        assertTrue(new QueryCachingPolicy.CacheOnLargeSegments(numDocs, Float.MIN_VALUE).shouldCache(query, ctx));
-        assertFalse(new QueryCachingPolicy.CacheOnLargeSegments(numDocs + 1, Float.MIN_VALUE).shouldCache(query, ctx));
-      }
-    }
-    reader.close();
-    w.close();
-    dir.close();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/81446cf3/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
index 5e50e58..70ed86d 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
@@ -1790,7 +1790,7 @@ public abstract class LuceneTestCase extends Assert {
   public static void overrideDefaultQueryCache() {
     // we need to reset the query cache in an @BeforeClass so that tests that
     // instantiate an IndexSearcher in an @BeforeClass method use a fresh new cache
-    IndexSearcher.setDefaultQueryCache(new LRUQueryCache(10000, 1 << 25));
+    IndexSearcher.setDefaultQueryCache(new LRUQueryCache(10000, 1 << 25, context -> true));
     IndexSearcher.setDefaultQueryCachingPolicy(MAYBE_CACHE_POLICY);
   }
 


[43/50] [abbrv] lucene-solr:solr-5750: SOLR-9037: replace multiple "/replication" strings with one static constant

Posted by ds...@apache.org.
SOLR-9037: replace multiple "/replication" strings with one static constant


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

Branch: refs/heads/solr-5750
Commit: 3b4ec73595adba4d1a4859a35be61f60e6a6be31
Parents: c7cdf28
Author: Christine Poerschke <cp...@apache.org>
Authored: Wed Apr 27 13:09:28 2016 +0100
Committer: Christine Poerschke <cp...@apache.org>
Committed: Wed Apr 27 13:09:28 2016 +0100

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  2 ++
 .../org/apache/solr/cloud/RecoveryStrategy.java |  6 ++---
 .../org/apache/solr/handler/IndexFetcher.java   | 12 +++++-----
 .../apache/solr/handler/ReplicationHandler.java |  2 ++
 .../solr/cloud/BasicDistributedZk2Test.java     |  3 ++-
 ...DistribDocExpirationUpdateProcessorTest.java |  3 ++-
 .../test/org/apache/solr/core/SolrCoreTest.java |  3 ++-
 .../apache/solr/handler/CheckBackupStatus.java  |  2 +-
 .../solr/handler/TestReplicationHandler.java    | 23 ++++++++++++--------
 .../handler/TestReplicationHandlerBackup.java   |  4 ++--
 .../apache/solr/handler/TestRestoreCore.java    |  2 +-
 .../TestRuleBasedAuthorizationPlugin.java       |  4 ++--
 12 files changed, 38 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b4ec735/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 4baea2c..57457bd 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -214,6 +214,8 @@ Other Changes
 
 * SOLR-8933: Solr should not close container streams. (Mike Drob, Uwe Schindler, Mark Miller)
 
+* SOLR-9037: Replace multiple "/replication" strings with one static constant. (Christine Poerschke)
+
 ==================  6.0.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/3b4ec735/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
index 61b3f90..abd00ae 100644
--- a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
+++ b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
@@ -72,8 +72,6 @@ public class RecoveryStrategy extends Thread implements Closeable {
   private static final int WAIT_FOR_UPDATES_WITH_STALE_STATE_PAUSE = Integer.getInteger("solr.cloud.wait-for-updates-with-stale-state-pause", 7000);
   private static final int MAX_RETRIES = 500;
   private static final int STARTING_RECOVERY_DELAY = 5000;
-  
-  private static final String REPLICATION_HANDLER = "/replication";
 
   public static interface RecoveryListener {
     public void recovered();
@@ -143,12 +141,12 @@ public class RecoveryStrategy extends Thread implements Closeable {
     commitOnLeader(leaderUrl);
     
     // use rep handler directly, so we can do this sync rather than async
-    SolrRequestHandler handler = core.getRequestHandler(REPLICATION_HANDLER);
+    SolrRequestHandler handler = core.getRequestHandler(ReplicationHandler.PATH);
     ReplicationHandler replicationHandler = (ReplicationHandler) handler;
     
     if (replicationHandler == null) {
       throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE,
-          "Skipping recovery, no " + REPLICATION_HANDLER + " handler found");
+          "Skipping recovery, no " + ReplicationHandler.PATH + " handler found");
     }
     
     ModifiableSolrParams solrParams = new ModifiableSolrParams();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b4ec735/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 71a2a84..8cdf132 100644
--- a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
+++ b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
@@ -192,9 +192,9 @@ public class IndexFetcher {
     if (masterUrl == null)
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
               "'masterUrl' is required for a slave");
-    if (masterUrl.endsWith("/replication")) {
+    if (masterUrl.endsWith(ReplicationHandler.PATH)) {
       masterUrl = masterUrl.substring(0, masterUrl.length()-12);
-      LOG.warn("'masterUrl' must be specified without the /replication suffix");
+      LOG.warn("'masterUrl' must be specified without the "+ReplicationHandler.PATH+" suffix");
     }
     this.masterUrl = masterUrl;
 
@@ -228,7 +228,7 @@ public class IndexFetcher {
     ModifiableSolrParams params = new ModifiableSolrParams();
     params.set(COMMAND, CMD_INDEX_VERSION);
     params.set(CommonParams.WT, JAVABIN);
-    params.set(CommonParams.QT, "/replication");
+    params.set(CommonParams.QT, ReplicationHandler.PATH);
     QueryRequest req = new QueryRequest(params);
 
     // TODO modify to use shardhandler
@@ -250,7 +250,7 @@ public class IndexFetcher {
     params.set(COMMAND,  CMD_GET_FILE_LIST);
     params.set(GENERATION, String.valueOf(gen));
     params.set(CommonParams.WT, JAVABIN);
-    params.set(CommonParams.QT, "/replication");
+    params.set(CommonParams.QT, ReplicationHandler.PATH);
     QueryRequest req = new QueryRequest(params);
 
     // TODO modify to use shardhandler
@@ -1597,7 +1597,7 @@ public class IndexFetcher {
 //    //the method is command=filecontent
       params.set(COMMAND, CMD_GET_FILE);
       params.set(GENERATION, Long.toString(indexGen));
-      params.set(CommonParams.QT, "/replication");
+      params.set(CommonParams.QT, ReplicationHandler.PATH);
       //add the version to download. This is used to reserve the download
       params.set(solrParamOutput, fileName);
       if (useInternalCompression) {
@@ -1729,7 +1729,7 @@ public class IndexFetcher {
     ModifiableSolrParams params = new ModifiableSolrParams();
     params.set(COMMAND, CMD_DETAILS);
     params.set("slave", false);
-    params.set(CommonParams.QT, "/replication");
+    params.set(CommonParams.QT, ReplicationHandler.PATH);
 
     // TODO use shardhandler
     try (HttpSolrClient client = new HttpSolrClient.Builder(masterUrl).withHttpClient(myHttpClient).build()) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b4ec735/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 267ab3d..c98fabf 100644
--- a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
@@ -117,6 +117,8 @@ import static org.apache.solr.common.params.CommonParams.NAME;
  */
 public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAware {
 
+  public static final String PATH = "/replication";
+
   private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   SolrCore core;
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b4ec735/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZk2Test.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZk2Test.java b/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZk2Test.java
index 61b9901..47d8212 100644
--- a/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZk2Test.java
+++ b/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZk2Test.java
@@ -42,6 +42,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.handler.CheckBackupStatus;
+import org.apache.solr.handler.ReplicationHandler;
 import org.junit.Test;
 
 /**
@@ -400,7 +401,7 @@ public class BasicDistributedZk2Test extends AbstractFullDistribZkTestBase {
     // try a backup command
     final HttpSolrClient client = (HttpSolrClient) shardToJetty.get(SHARD2).get(0).client.solrClient;
     ModifiableSolrParams params = new ModifiableSolrParams();
-    params.set("qt", "/replication");
+    params.set("qt", ReplicationHandler.PATH);
     params.set("command", "backup");
     Path location = createTempDir();
     location = FilterPath.unwrap(location).toRealPath();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b4ec735/solr/core/src/test/org/apache/solr/cloud/DistribDocExpirationUpdateProcessorTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/DistribDocExpirationUpdateProcessorTest.java b/solr/core/src/test/org/apache/solr/cloud/DistribDocExpirationUpdateProcessorTest.java
index 6be317f..eaf2460 100644
--- a/solr/core/src/test/org/apache/solr/cloud/DistribDocExpirationUpdateProcessorTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/DistribDocExpirationUpdateProcessorTest.java
@@ -20,6 +20,7 @@ import org.apache.lucene.util.LuceneTestCase.Slow;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.handler.ReplicationHandler;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 
@@ -140,7 +141,7 @@ public class DistribDocExpirationUpdateProcessorTest extends AbstractFullDistrib
         ModifiableSolrParams params = new ModifiableSolrParams();
         params.set("command","indexversion");
         params.set("_trace","getIndexVersion");
-        params.set("qt","/replication");
+        params.set("qt",ReplicationHandler.PATH);
         QueryRequest req = new QueryRequest(params);
     
         NamedList<Object> res = replicaRunner.client.solrClient.request(req);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b4ec735/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java b/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java
index 200935a..9e6029a 100644
--- a/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java
+++ b/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java
@@ -18,6 +18,7 @@ package org.apache.solr.core;
 
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.handler.ReplicationHandler;
 import org.apache.solr.handler.RequestHandlerBase;
 import org.apache.solr.handler.component.QueryComponent;
 import org.apache.solr.handler.component.SpellCheckComponent;
@@ -96,7 +97,7 @@ public class SolrCoreTest extends SolrTestCaseJ4 {
       ++ihCount; assertEquals(pathToClassMap.get("/config"), "solr.SolrConfigHandler");
       ++ihCount; assertEquals(pathToClassMap.get("/export"), "solr.SearchHandler");
       ++ihCount; assertEquals(pathToClassMap.get("/get"), "solr.RealTimeGetHandler");
-      ++ihCount; assertEquals(pathToClassMap.get("/replication"), "solr.ReplicationHandler");
+      ++ihCount; assertEquals(pathToClassMap.get(ReplicationHandler.PATH), "solr.ReplicationHandler");
       ++ihCount; assertEquals(pathToClassMap.get("/schema"), "solr.SchemaHandler");
       ++ihCount; assertEquals(pathToClassMap.get("/sql"), "solr.SQLHandler");
       ++ihCount; assertEquals(pathToClassMap.get("/stream"), "solr.StreamHandler");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b4ec735/solr/core/src/test/org/apache/solr/handler/CheckBackupStatus.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/CheckBackupStatus.java b/solr/core/src/test/org/apache/solr/handler/CheckBackupStatus.java
index ab1d553..706a2fe 100644
--- a/solr/core/src/test/org/apache/solr/handler/CheckBackupStatus.java
+++ b/solr/core/src/test/org/apache/solr/handler/CheckBackupStatus.java
@@ -44,7 +44,7 @@ public class CheckBackupStatus extends SolrTestCaseJ4 {
   }
 
   public void fetchStatus() throws IOException {
-    String masterUrl = client.getBaseURL() + "/replication?command=" + ReplicationHandler.CMD_DETAILS;
+    String masterUrl = client.getBaseURL() + ReplicationHandler.PATH + "?command=" + ReplicationHandler.CMD_DETAILS;
     response = client.getHttpClient().execute(new HttpGet(masterUrl), new BasicResponseHandler());
     if(pException.matcher(response).find()) {
       fail("Failed to create backup");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b4ec735/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 2b974cb..cf40d4b 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
@@ -210,7 +210,7 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
     ModifiableSolrParams params = new ModifiableSolrParams();
     params.set("command","details");
     params.set("_trace","getDetails");
-    params.set("qt","/replication");
+    params.set("qt",ReplicationHandler.PATH);
     QueryRequest req = new QueryRequest(params);
 
     NamedList<Object> res = s.request(req);
@@ -231,7 +231,7 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
     ModifiableSolrParams params = new ModifiableSolrParams();
     params.set("command","commits");
     params.set("_trace","getCommits");
-    params.set("qt","/replication");
+    params.set("qt",ReplicationHandler.PATH);
     QueryRequest req = new QueryRequest(params);
 
     NamedList<Object> res = s.request(req);
@@ -247,7 +247,7 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
     ModifiableSolrParams params = new ModifiableSolrParams();
     params.set("command","indexversion");
     params.set("_trace","getIndexVersion");
-    params.set("qt","/replication");
+    params.set("qt",ReplicationHandler.PATH);
     QueryRequest req = new QueryRequest(params);
 
     NamedList<Object> res = s.request(req);
@@ -280,6 +280,11 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
   }
 
   @Test
+  public void doTestHandlerPathUnchanged() throws Exception {
+    assertEquals("/replication", ReplicationHandler.PATH);
+  }
+
+  @Test
   public void doTestDetails() throws Exception {
     clearIndexWithReplication();
     { 
@@ -451,7 +456,7 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
   //jetty servers.
   private void invokeReplicationCommand(int pJettyPort, String pCommand) throws IOException
   {
-    String masterUrl = buildUrl(pJettyPort) + "/" + DEFAULT_TEST_CORENAME + "/replication?command=" + pCommand;
+    String masterUrl = buildUrl(pJettyPort) + "/" + DEFAULT_TEST_CORENAME + ReplicationHandler.PATH+"?command=" + pCommand;
     URL u = new URL(masterUrl);
     InputStream stream = u.openStream();
     stream.close();
@@ -610,8 +615,8 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
     assertEquals(nDocs, masterQueryResult.getNumFound());
 
     // index fetch
-    String masterUrl = buildUrl(slaveJetty.getLocalPort()) + "/" + DEFAULT_TEST_CORENAME + "/replication?command=fetchindex&masterUrl=";
-    masterUrl += buildUrl(masterJetty.getLocalPort()) + "/" + DEFAULT_TEST_CORENAME + "/replication";
+    String masterUrl = buildUrl(slaveJetty.getLocalPort()) + "/" + DEFAULT_TEST_CORENAME + ReplicationHandler.PATH+"?command=fetchindex&masterUrl=";
+    masterUrl += buildUrl(masterJetty.getLocalPort()) + "/" + DEFAULT_TEST_CORENAME + ReplicationHandler.PATH;
     URL url = new URL(masterUrl);
     InputStream stream = url.openStream();
     stream.close();
@@ -925,7 +930,7 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
     
     // check vs /replication?command=indexversion call
     ModifiableSolrParams params = new ModifiableSolrParams();
-    params.set("qt", "/replication");
+    params.set("qt", ReplicationHandler.PATH);
     params.set("_trace", "assertVersions");
     params.set("command", "indexversion");
     QueryRequest req = new QueryRequest(params);
@@ -964,9 +969,9 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
     InputStream stream;
     masterUrl = buildUrl(to.getLocalPort())
         + "/" + DEFAULT_TEST_CORENAME
-        + "/replication?wait=true&command=fetchindex&masterUrl="
+        + ReplicationHandler.PATH+"?wait=true&command=fetchindex&masterUrl="
         + buildUrl(from.getLocalPort())
-        + "/" + DEFAULT_TEST_CORENAME + "/replication";
+        + "/" + DEFAULT_TEST_CORENAME + ReplicationHandler.PATH;
     url = new URL(masterUrl);
     stream = url.openStream();
     stream.close();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b4ec735/solr/core/src/test/org/apache/solr/handler/TestReplicationHandlerBackup.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/TestReplicationHandlerBackup.java b/solr/core/src/test/org/apache/solr/handler/TestReplicationHandlerBackup.java
index 9779c78..bfad782 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestReplicationHandlerBackup.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestReplicationHandlerBackup.java
@@ -272,7 +272,7 @@ public class TestReplicationHandlerBackup extends SolrJettyTestBase {
 
   public static void runBackupCommand(JettySolrRunner masterJetty, String cmd, String params) throws IOException {
     String masterUrl = buildUrl(masterJetty.getLocalPort(), context) + "/" + DEFAULT_TEST_CORENAME
-        + "/replication?command=" + cmd + params;
+        + ReplicationHandler.PATH+"?command=" + cmd + params;
     InputStream stream = null;
     try {
       URL url = new URL(masterUrl);
@@ -295,7 +295,7 @@ public class TestReplicationHandlerBackup extends SolrJettyTestBase {
     }
 
     public boolean fetchStatus() throws IOException {
-      String masterUrl = buildUrl(masterJetty.getLocalPort(), context) + "/" + DEFAULT_TEST_CORENAME + "/replication?command=" + ReplicationHandler.CMD_DETAILS;
+      String masterUrl = buildUrl(masterJetty.getLocalPort(), context) + "/" + DEFAULT_TEST_CORENAME + ReplicationHandler.PATH + "?command=" + ReplicationHandler.CMD_DETAILS;
       URL url;
       InputStream stream = null;
       try {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b4ec735/solr/core/src/test/org/apache/solr/handler/TestRestoreCore.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/TestRestoreCore.java b/solr/core/src/test/org/apache/solr/handler/TestRestoreCore.java
index 57eb2cf..db5fc7c 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestRestoreCore.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestRestoreCore.java
@@ -230,7 +230,7 @@ public class TestRestoreCore extends SolrJettyTestBase {
 
   private boolean fetchRestoreStatus() throws IOException {
     String masterUrl = buildUrl(masterJetty.getLocalPort(), context) + "/" + DEFAULT_TEST_CORENAME +
-        "/replication?command=" + ReplicationHandler.CMD_RESTORE_STATUS;
+        ReplicationHandler.PATH + "?command=" + ReplicationHandler.CMD_RESTORE_STATUS;
     final Pattern pException = Pattern.compile("<str name=\"exception\">(.*?)</str>");
 
     InputStream stream = null;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b4ec735/solr/core/src/test/org/apache/solr/security/TestRuleBasedAuthorizationPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/TestRuleBasedAuthorizationPlugin.java b/solr/core/src/test/org/apache/solr/security/TestRuleBasedAuthorizationPlugin.java
index 5abf21c..628b4a8 100644
--- a/solr/core/src/test/org/apache/solr/security/TestRuleBasedAuthorizationPlugin.java
+++ b/solr/core/src/test/org/apache/solr/security/TestRuleBasedAuthorizationPlugin.java
@@ -190,14 +190,14 @@ public class TestRuleBasedAuthorizationPlugin extends SolrTestCaseJ4 {
     ((Map)rules.get("user-role")).put("cio","su");
     ((List)rules.get("permissions")).add( makeMap("name", "all", "role", "su"));
 
-    checkRules(makeMap("resource", "/replication",
+    checkRules(makeMap("resource", ReplicationHandler.PATH,
         "httpMethod", "POST",
         "userPrincipal", "tim",
         "handler", new ReplicationHandler(),
         "collectionRequests", singletonList(new CollectionRequest("mycoll")) )
         , FORBIDDEN, rules);
 
-    checkRules(makeMap("resource", "/replication",
+    checkRules(makeMap("resource", ReplicationHandler.PATH,
         "httpMethod", "POST",
         "userPrincipal", "cio",
         "handler", new ReplicationHandler(),


[48/50] [abbrv] lucene-solr:solr-5750: SOLR-5750: move location of backup.properties and rename collection state.json

Posted by ds...@apache.org.
SOLR-5750: move location of backup.properties and rename collection state.json


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

Branch: refs/heads/solr-5750
Commit: cf77521a55e721ee8da56de091d883092848093c
Parents: 42b6ed4
Author: David Smiley <ds...@apache.org>
Authored: Wed Apr 27 11:46:10 2016 -0400
Committer: David Smiley <ds...@apache.org>
Committed: Wed Apr 27 11:46:10 2016 -0400

----------------------------------------------------------------------
 .../apache/solr/cloud/OverseerCollectionMessageHandler.java | 9 ++++-----
 1 file changed, 4 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cf77521a/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java b/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
index e1ea1dc..1966116 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
@@ -340,12 +340,12 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
     Path backupZkPath =  backupPath.resolve("zk_backup");
 
     Properties properties = new Properties();
-    try (Reader in = Files.newBufferedReader(backupZkPath.resolve("backup.properties"), StandardCharsets.UTF_8)) {
+    try (Reader in = Files.newBufferedReader(backupPath.resolve("backup.properties"), StandardCharsets.UTF_8)) {
       properties.load(in);
     }
 
     String backupCollection = (String) properties.get("collection");
-    byte[] data = Files.readAllBytes(backupZkPath.resolve("collection_state_backup.json"));
+    byte[] data = Files.readAllBytes(backupZkPath.resolve("collection_state.json"));
     ClusterState backupClusterState = ClusterState.load(-1, data, Collections.emptySet());
     DocCollection backupCollectionState = backupClusterState.getCollection(backupCollection);
 
@@ -546,11 +546,10 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
     //Save the collection's state. Can be part of the monolithic clusterstate.json or a individual state.json
     //Since we don't want to distinguish we extract the state and back it up as a separate json
     DocCollection collection = zkStateReader.getClusterState().getCollection(collectionName);
-    Files.write(zkBackup.resolve("collection_state_backup.json"),//nocommit or simply clusterstate.json?
+    Files.write(zkBackup.resolve("collection_state.json"),
         Utils.toJSON(Collections.singletonMap(collectionName, collection)));
 
-    //nocommit why is it stored in zk_backup; shouldn't it be in backupPath?
-    Path propertiesPath = zkBackup.resolve("backup.properties");
+    Path propertiesPath = backupPath.resolve("backup.properties");
     Properties properties = new Properties();
 
     properties.put("snapshotName", backupName);


[34/50] [abbrv] lucene-solr:solr-5750: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/lucene-solr

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


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

Branch: refs/heads/solr-5750
Commit: 506d61abe773ccf0b9aabf6eb88c83380c1eda95
Parents: 0233765 6fa5166
Author: Noble Paul <no...@apache.org>
Authored: Tue Apr 26 19:24:52 2016 +0530
Committer: Noble Paul <no...@apache.org>
Committed: Tue Apr 26 19:24:52 2016 +0530

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  13 +-
 .../org/apache/lucene/document/StringField.java |   4 +-
 .../src/java/org/apache/lucene/geo/Polygon.java | 221 ---------
 .../java/org/apache/lucene/geo/Polygon2D.java   | 473 +++++++++++++++++++
 .../org/apache/lucene/search/LRUQueryCache.java | 238 ++++++----
 .../lucene/search/QueryCachingPolicy.java       |  18 +-
 .../search/UsageTrackingQueryCachingPolicy.java |   7 +-
 .../test/org/apache/lucene/geo/TestPolygon.java | 330 -------------
 .../org/apache/lucene/geo/TestPolygon2D.java    | 289 +++++++++++
 .../apache/lucene/search/TestIndexSearcher.java |   2 +-
 .../apache/lucene/search/TestLRUQueryCache.java |   8 +-
 .../apache/lucene/search/TestPointQueries.java  |   2 +-
 .../TestUsageTrackingFilterCachingPolicy.java   |  17 +-
 .../apache/lucene/queries/TermsQueryTest.java   |  16 +-
 .../lucene/queryparser/xml/TestCoreParser.java  |   9 +-
 .../lucene/document/LatLonDocValuesField.java   | 135 ++++++
 .../org/apache/lucene/document/LatLonGrid.java  | 168 -------
 .../org/apache/lucene/document/LatLonPoint.java |  68 +--
 .../lucene/document/LatLonPointBoxQuery.java    | 287 +++++++++++
 .../document/LatLonPointDistanceComparator.java |   2 +-
 .../document/LatLonPointDistanceQuery.java      |  17 +-
 .../document/LatLonPointInPolygonQuery.java     |  34 +-
 .../org/apache/lucene/document/LatLonTree.java  | 401 ----------------
 .../apache/lucene/document/MatchingPoints.java  |  90 ++++
 .../document/TestLatLonDocValuesField.java      |  30 ++
 .../apache/lucene/document/TestLatLonGrid.java  | 106 -----
 .../apache/lucene/document/TestLatLonPoint.java |   3 -
 .../document/TestLatLonPointDistanceSort.java   |  20 +-
 .../apache/lucene/document/TestLatLonTree.java  |  53 ---
 .../org/apache/lucene/document/TestNearest.java |   3 +-
 .../search/GeoPointInPolygonQueryImpl.java      |  16 +-
 .../org/apache/lucene/spatial3d/Geo3DPoint.java |  17 +-
 .../spatial3d/PointInShapeIntersectVisitor.java |   5 +
 .../spatial3d/geom/GeoConcavePolygon.java       |  66 ++-
 .../lucene/spatial3d/geom/GeoConvexPolygon.java |  66 ++-
 .../spatial3d/geom/GeoPolygonFactory.java       |   6 +-
 .../lucene/spatial3d/geom/GeoStandardPath.java  |   4 +-
 .../apache/lucene/geo/BaseGeoPointTestCase.java |   2 +-
 .../java/org/apache/lucene/geo/GeoTestUtil.java |  54 +++
 .../org/apache/lucene/index/RandomCodec.java    |   2 +-
 .../org/apache/lucene/util/LuceneTestCase.java  |   2 +-
 solr/CHANGES.txt                                |   5 +
 .../src/java/org/apache/solr/cloud/Assign.java  |   8 +-
 .../OverseerAutoReplicaFailoverThread.java      |   1 +
 .../cloud/OverseerCollectionMessageHandler.java |  40 +-
 .../cloud/OverseerConfigSetMessageHandler.java  |   1 +
 .../cloud/overseer/ClusterStateMutator.java     |  16 +-
 .../solr/cloud/overseer/CollectionMutator.java  |   6 +-
 .../apache/solr/cloud/overseer/NodeMutator.java |   9 +-
 .../solr/cloud/overseer/ReplicaMutator.java     |  36 +-
 .../solr/cloud/overseer/SliceMutator.java       |  39 +-
 .../solr/cloud/overseer/ZkStateWriter.java      |   3 +-
 .../src/java/org/apache/solr/core/SolrCore.java |   5 +-
 .../solr/handler/CdcrUpdateLogSynchronizer.java |   4 +-
 .../org/apache/solr/handler/SQLHandler.java     |   1 +
 .../solr/handler/admin/CollectionsHandler.java  |  27 +-
 .../solr/handler/admin/CoreAdminOperation.java  |  11 +-
 .../org/apache/solr/servlet/HttpSolrCall.java   |  11 +-
 .../processor/DistributedUpdateProcessor.java   |   9 +-
 .../src/java/org/apache/solr/util/SolrCLI.java  |   1 +
 .../org/apache/solr/util/SolrLogLayout.java     |  10 +-
 .../test/org/apache/solr/cloud/AssignTest.java  |   2 +-
 .../cloud/CollectionsAPIDistributedZkTest.java  |   8 +-
 .../solr/cloud/CollectionsAPISolrJTest.java     |   2 +-
 .../org/apache/solr/cloud/OverseerTest.java     |   3 +-
 .../test/org/apache/solr/core/SolrCoreTest.java |  40 ++
 .../org/apache/solr/handler/TestSQLHandler.java |   1 +
 .../client/solrj/io/stream/TopicStream.java     |   1 +
 .../apache/solr/common/cloud/ClusterState.java  |  48 +-
 .../solr/common/cloud/ClusterStateUtil.java     |   1 +
 .../apache/solr/common/cloud/DocCollection.java |   6 +
 .../apache/solr/common/cloud/ZkStateReader.java |  12 +-
 .../solr/client/solrj/io/sql/JdbcTest.java      |   1 +
 73 files changed, 1941 insertions(+), 1731 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/506d61ab/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
----------------------------------------------------------------------


[11/50] [abbrv] lucene-solr:solr-5750: LUCENE-7239: Use interval tree to speed up LatLonPoint.newPolygonQuery

Posted by ds...@apache.org.
LUCENE-7239: Use interval tree to speed up LatLonPoint.newPolygonQuery


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

Branch: refs/heads/solr-5750
Commit: 4fd5d88080e70efd8b944198c9798d30d6f94a15
Parents: f3de223
Author: Robert Muir <rm...@apache.org>
Authored: Thu Apr 21 20:14:31 2016 -0400
Committer: Robert Muir <rm...@apache.org>
Committed: Thu Apr 21 20:15:33 2016 -0400

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   7 +-
 .../org/apache/lucene/document/LatLonGrid.java  |  26 +-
 .../document/LatLonPointInPolygonQuery.java     |  61 +--
 .../org/apache/lucene/document/LatLonTree.java  | 401 +++++++++++++++++++
 .../apache/lucene/document/TestLatLonTree.java  |  53 +++
 5 files changed, 476 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fd5d880/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index da5a345..c2561b3 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -48,9 +48,6 @@ Optimizations
 * LUCENE-7105, LUCENE-7215: Optimize LatLonPoint's newDistanceQuery. 
   (Robert Muir)
 
-* LUCENE-7109: LatLonPoint's newPolygonQuery supports two-phase 
-  iteration. (Robert Muir)
-
 * LUCENE-7097: IntroSorter now recurses to 2 * log_2(count) quicksort
   stack depth before switching to heapsort (Adrien Grand, Mike McCandless)
 
@@ -64,8 +61,8 @@ Optimizations
   multiple polygons and holes, with memory usage independent of
   polygon complexity. (Karl Wright, Mike McCandless, Robert Muir)
 
-* LUCENE-7159, LUCENE-7222, LUCENE-7229: Speed up LatLonPoint polygon performance for complex
-  polygons. (Robert Muir)
+* LUCENE-7159, LUCENE-7222, LUCENE-7229, LUCENE-7239: Speed up LatLonPoint 
+  polygon performance. (Robert Muir)
 
 * LUCENE-7211: Reduce memory & GC for spatial RPT Intersects when the number of
   matching docs is small. (Jeff Wartes, David Smiley)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fd5d880/lucene/sandbox/src/java/org/apache/lucene/document/LatLonGrid.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonGrid.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonGrid.java
index db62729..2083f03 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonGrid.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonGrid.java
@@ -60,14 +60,14 @@ final class LatLonGrid {
   final long latPerCell;
   final long lonPerCell;
   
-  final Polygon[] polygons;
+  final LatLonTree[] tree;
   
   LatLonGrid(int minLat, int maxLat, int minLon, int maxLon, Polygon... polygons) {
     this.minLat = minLat;
     this.maxLat = maxLat;
     this.minLon = minLon;
     this.maxLon = maxLon;
-    this.polygons = polygons;
+    this.tree = LatLonTree.build(polygons);
     if (minLon > maxLon) {
       // maybe make 2 grids if you want this? 
       throw new IllegalArgumentException("Grid cannot cross the dateline");
@@ -88,12 +88,12 @@ final class LatLonGrid {
       // but it prevents edge case bugs.
       latPerCell = latitudeRange / (GRID_SIZE - 1);
       lonPerCell = longitudeRange / (GRID_SIZE - 1);
-      fill(polygons, 0, GRID_SIZE, 0, GRID_SIZE);
+      fill(0, GRID_SIZE, 0, GRID_SIZE);
     }
   }
   
   /** fills a 2D range of grid cells [minLatIndex .. maxLatIndex) X [minLonIndex .. maxLonIndex) */
-  void fill(Polygon[] polygons, int minLatIndex, int maxLatIndex, int minLonIndex, int maxLonIndex) {
+  void fill(int minLatIndex, int maxLatIndex, int minLonIndex, int maxLonIndex) {
     // grid cells at the edge of the bounding box are typically smaller than normal, because we spill over.
     long cellMinLat = minLat + (minLatIndex * latPerCell);
     long cellMaxLat = Math.min(maxLat, minLat + (maxLatIndex * latPerCell) - 1);
@@ -104,10 +104,10 @@ final class LatLonGrid {
     assert cellMaxLat >= cellMinLat;
     assert cellMaxLon >= cellMinLon;
 
-    Relation relation = Polygon.relate(polygons, decodeLatitude((int) cellMinLat),
-                                                 decodeLatitude((int) cellMaxLat),
-                                                 decodeLongitude((int) cellMinLon),
-                                                 decodeLongitude((int) cellMaxLon));
+    Relation relation = LatLonTree.relate(tree, decodeLatitude((int) cellMinLat),
+                                                decodeLatitude((int) cellMaxLat),
+                                                decodeLongitude((int) cellMinLon),
+                                                decodeLongitude((int) cellMaxLon));
     if (relation != Relation.CELL_CROSSES_QUERY) {
       // we know the answer for this region, fill the cell range
       for (int i = minLatIndex; i < maxLatIndex; i++) {
@@ -127,10 +127,10 @@ final class LatLonGrid {
       // grid range crosses our polygon, keep recursing.
       int midLatIndex = (minLatIndex + maxLatIndex) >>> 1;
       int midLonIndex = (minLonIndex + maxLonIndex) >>> 1;
-      fill(polygons, minLatIndex, midLatIndex, minLonIndex, midLonIndex);
-      fill(polygons, minLatIndex, midLatIndex, midLonIndex, maxLonIndex);
-      fill(polygons, midLatIndex, maxLatIndex, minLonIndex, midLonIndex);
-      fill(polygons, midLatIndex, maxLatIndex, midLonIndex, maxLonIndex);
+      fill(minLatIndex, midLatIndex, minLonIndex, midLonIndex);
+      fill(minLatIndex, midLatIndex, midLonIndex, maxLonIndex);
+      fill(midLatIndex, maxLatIndex, minLonIndex, midLonIndex);
+      fill(midLatIndex, maxLatIndex, midLonIndex, maxLonIndex);
     }
   }
   
@@ -147,7 +147,7 @@ final class LatLonGrid {
     // the grid is unsure (boundary): do a real test.
     double docLatitude = decodeLatitude(latitude);
     double docLongitude = decodeLongitude(longitude);
-    return Polygon.contains(polygons, docLatitude, docLongitude);
+    return LatLonTree.contains(tree, docLatitude, docLongitude);
   }
   
   /** Returns grid index of lat/lon, or -1 if the value is outside of the bounding box. */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fd5d880/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java
index 7c504e2..15361b5 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java
@@ -29,19 +29,13 @@ 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.TwoPhaseIterator;
 import org.apache.lucene.search.Weight;
 import org.apache.lucene.index.PointValues;
-import org.apache.lucene.index.SortedNumericDocValues;
-import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.util.BitSet;
 import org.apache.lucene.util.DocIdSetBuilder;
-import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.NumericUtils;
-import org.apache.lucene.util.SparseFixedBitSet;
 import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.geo.Polygon;
 
@@ -98,13 +92,6 @@ final class LatLonPointInPolygonQuery extends Query {
     NumericUtils.intToSortableBytes(encodeLongitude(box.minLon), minLon, 0);
     NumericUtils.intToSortableBytes(encodeLongitude(box.maxLon), maxLon, 0);
 
-    // TODO: make this fancier, but currently linear with number of vertices
-    float cumulativeCost = 0;
-    for (Polygon polygon : polygons) {
-      cumulativeCost += 20 * (polygon.getPolyLats().length + polygon.getHoles().length);
-    }
-    final float matchCost = cumulativeCost;
-
     final LatLonGrid grid = new LatLonGrid(encodeLatitude(box.minLat),
                                            encodeLatitude(box.maxLat),
                                            encodeLongitude(box.minLon),
@@ -127,22 +114,14 @@ final class LatLonPointInPolygonQuery extends Query {
         }
         LatLonPoint.checkCompatible(fieldInfo);
 
-        // approximation (postfiltering has not yet been applied)
+        // matching docids
         DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc());
-        // subset of documents that need no postfiltering, this is purely an optimization
-        final BitSet preApproved;
-        // dumb heuristic: if the field is really sparse, use a sparse impl
-        if (values.getDocCount(field) * 100L < reader.maxDoc()) {
-          preApproved = new SparseFixedBitSet(reader.maxDoc());
-        } else {
-          preApproved = new FixedBitSet(reader.maxDoc());
-        }
+
         values.intersect(field, 
                          new IntersectVisitor() {
                            @Override
                            public void visit(int docID) {
                              result.add(docID);
-                             preApproved.set(docID);
                            }
 
                            @Override
@@ -156,7 +135,10 @@ final class LatLonPointInPolygonQuery extends Query {
                                // outside of global bounding box range
                                return;
                              }
-                             result.add(docID);
+                             if (grid.contains(NumericUtils.sortableBytesToInt(packedValue, 0), 
+                                               NumericUtils.sortableBytesToInt(packedValue, Integer.BYTES))) {
+                               result.add(docID);
+                             }
                            }
 
                            @Override
@@ -184,36 +166,7 @@ final class LatLonPointInPolygonQuery extends Query {
           return null;
         }
 
-        // return two-phase iterator using docvalues to postfilter candidates
-        SortedNumericDocValues docValues = DocValues.getSortedNumeric(reader, field);
-
-        TwoPhaseIterator iterator = new TwoPhaseIterator(disi) {
-          @Override
-          public boolean matches() throws IOException {
-            int docId = disi.docID();
-            if (preApproved.get(docId)) {
-              return true;
-            } else {
-              docValues.setDocument(docId);
-              int count = docValues.count();
-              for (int i = 0; i < count; i++) {
-                long encoded = docValues.valueAt(i);
-                int latitudeBits = (int)(encoded >> 32);
-                int longitudeBits = (int)(encoded & 0xFFFFFFFF);
-                if (grid.contains(latitudeBits, longitudeBits)) {
-                  return true;
-                }
-              }
-              return false;
-            }
-          }
-
-          @Override
-          public float matchCost() {
-            return matchCost;
-          }
-        };
-        return new ConstantScoreScorer(this, score(), iterator);
+        return new ConstantScoreScorer(this, score(), disi);
       }
     };
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fd5d880/lucene/sandbox/src/java/org/apache/lucene/document/LatLonTree.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonTree.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonTree.java
new file mode 100644
index 0000000..8a6e6d8
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonTree.java
@@ -0,0 +1,401 @@
+/*
+ * 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.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.lucene.geo.Polygon;
+import org.apache.lucene.index.PointValues.Relation;
+
+/**
+ * 2D polygon implementation represented as a randomized interval tree of edges.
+ * <p>
+ * contains() and crosses() are still O(n), but for most practical polygons 
+ * are much faster than brute force.
+ * <p>
+ * Loosely based on the algorithm described in <a href="http://www-ma2.upc.es/geoc/Schirra-pointPolygon.pdf">
+ * http://www-ma2.upc.es/geoc/Schirra-pointPolygon.pdf</a>.
+ */
+// Both Polygon.contains() and Polygon.crossesSlowly() loop all edges, and first check that the edge is within a range.
+// we just organize the edges to do the same computations on the same subset of edges more efficiently. 
+// TODO: clean this up, call it Polygon2D, and remove all the 2D methods from Polygon?
+final class LatLonTree {
+  private final LatLonTree[] holes;
+
+  /** minimum latitude of this polygon's bounding box area */
+  final double minLat;
+  /** maximum latitude of this polygon's bounding box area */
+  final double maxLat;
+  /** minimum longitude of this polygon's bounding box area */
+  final double minLon;
+  /** maximum longitude of this polygon's bounding box area */
+  final double maxLon;
+  
+  /** root node of our tree */
+  final Edge tree;
+
+  // TODO: "pack" all the gons and holes into one tree with separator.
+  // the algorithms support this, but we have to be careful.
+  LatLonTree(Polygon polygon, LatLonTree... holes) {
+    this.holes = holes.clone();
+    this.minLat = polygon.minLat;
+    this.maxLat = polygon.maxLat;
+    this.minLon = polygon.minLon;
+    this.maxLon = polygon.maxLon;
+    
+    // create interval tree of edges
+    this.tree = createTree(polygon.getPolyLats(), polygon.getPolyLons());
+  }
+
+  /** 
+   * Returns true if the point is contained within this polygon.
+   * <p>
+   * See <a href="https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html">
+   * https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html</a> for more information.
+   */
+  boolean contains(double latitude, double longitude) {
+    // check bounding box
+    if (latitude < minLat || latitude > maxLat || longitude < minLon || longitude > maxLon) {
+      return false;
+    }
+    
+    if (tree.contains(latitude, longitude)) {
+      for (LatLonTree hole : holes) {
+        if (hole.contains(latitude, longitude)) {
+          return false;
+        }
+      }
+      return true;
+    }
+    
+    return false;
+  }
+  
+  /** Returns relation to the provided rectangle */
+  Relation relate(double minLat, double maxLat, double minLon, double maxLon) {
+    // if the bounding boxes are disjoint then the shape does not cross
+    if (maxLon < this.minLon || minLon > this.maxLon || maxLat < this.minLat || minLat > this.maxLat) {
+      return Relation.CELL_OUTSIDE_QUERY;
+    }
+    // if the rectangle fully encloses us, we cross.
+    if (minLat <= this.minLat && maxLat >= this.maxLat && minLon <= this.minLon && maxLon >= this.maxLon) {
+      return Relation.CELL_CROSSES_QUERY;
+    }
+    // check any holes
+    for (LatLonTree hole : holes) {
+      Relation holeRelation = hole.relate(minLat, maxLat, minLon, maxLon);
+      if (holeRelation == Relation.CELL_CROSSES_QUERY) {
+        return Relation.CELL_CROSSES_QUERY;
+      } else if (holeRelation == Relation.CELL_INSIDE_QUERY) {
+        return Relation.CELL_OUTSIDE_QUERY;
+      }
+    }
+    // check each corner: if < 4 are present, its cheaper than crossesSlowly
+    int numCorners = numberOfCorners(minLat, maxLat, minLon, maxLon);
+    if (numCorners == 4) {
+      if (tree.crosses(minLat, maxLat, minLon, maxLon)) {
+        return Relation.CELL_CROSSES_QUERY;
+      }
+      return Relation.CELL_INSIDE_QUERY;
+    } else if (numCorners > 0) {
+      return Relation.CELL_CROSSES_QUERY;
+    }
+    
+    // we cross
+    if (tree.crosses(minLat, maxLat, minLon, maxLon)) {
+      return Relation.CELL_CROSSES_QUERY;
+    }
+    
+    return Relation.CELL_OUTSIDE_QUERY;
+  }
+  
+  // returns 0, 4, or something in between
+  private int numberOfCorners(double minLat, double maxLat, double minLon, double maxLon) {
+    int containsCount = 0;
+    if (contains(minLat, minLon)) {
+      containsCount++;
+    }
+    if (contains(minLat, maxLon)) {
+      containsCount++;
+    }
+    if (containsCount == 1) {
+      return containsCount;
+    }
+    if (contains(maxLat, maxLon)) {
+      containsCount++;
+    }
+    if (containsCount == 2) {
+      return containsCount;
+    }
+    if (contains(maxLat, minLon)) {
+      containsCount++;
+    }
+    return containsCount;
+  }
+
+  /** Helper for multipolygon logic: returns true if any of the supplied polygons contain the point */
+  static boolean contains(LatLonTree[] polygons, double latitude, double longitude) {
+    for (LatLonTree polygon : polygons) {
+      if (polygon.contains(latitude, longitude)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /** Returns the multipolygon relation for the rectangle */
+  static Relation relate(LatLonTree[] polygons, double minLat, double maxLat, double minLon, double maxLon) {
+    for (LatLonTree polygon : polygons) {
+      Relation relation = polygon.relate(minLat, maxLat, minLon, maxLon);
+      if (relation != Relation.CELL_OUTSIDE_QUERY) {
+        // note: we optimize for non-overlapping multipolygons. so if we cross one,
+        // we won't keep iterating to try to find a contains.
+        return relation;
+      }
+    }
+    return Relation.CELL_OUTSIDE_QUERY;
+  }
+  
+  /** Builds a tree from multipolygon */
+  static LatLonTree[] build(Polygon... polygons) {
+    // TODO: use one tree with separators (carefully!)
+    LatLonTree trees[] = new LatLonTree[polygons.length];
+    for (int i = 0; i < trees.length; i++) {
+      Polygon gon = polygons[i];
+      Polygon gonHoles[] = gon.getHoles();
+      LatLonTree holes[] = new LatLonTree[gonHoles.length];
+      for (int j = 0; j < holes.length; j++) {
+        holes[j] = new LatLonTree(gonHoles[j]);
+      }
+      trees[i] = new LatLonTree(gon, holes);
+    }
+    return trees;
+  }
+  
+  /** 
+   * Internal tree node: represents polygon edge from lat1,lon1 to lat2,lon2.
+   * The sort value is {@code low}, which is the minimum latitude of the edge.
+   * {@code max} stores the maximum latitude of this edge or any children.
+   */
+  static final class Edge {
+    // lat-lon pair (in original order) of the two vertices
+    final double lat1, lat2;
+    final double lon1, lon2;
+    /** min of this edge */
+    final double low;
+    /** max latitude of this edge or any children */
+    double max;
+    
+    /** left child edge, or null */
+    Edge left;
+    /** right child edge, or null */
+    Edge right;
+
+    Edge(double lat1, double lon1, double lat2, double lon2, double low, double max) {
+      this.lat1 = lat1;
+      this.lon1 = lon1;
+      this.lat2 = lat2;
+      this.lon2 = lon2;
+      this.low = low;
+      this.max = max;
+    }
+    
+    /** 
+     * Returns true if the point crosses this edge subtree an odd number of times
+     * <p>
+     * See <a href="https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html">
+     * https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html</a> for more information.
+     */
+    // ported to java from https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html
+    // original code under the BSD license (https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html#License%20to%20Use)
+    //
+    // Copyright (c) 1970-2003, Wm. Randolph Franklin
+    //
+    // Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated 
+    // documentation files (the "Software"), to deal in the Software without restriction, including without limitation 
+    // the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and 
+    // to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+    //
+    // 1. Redistributions of source code must retain the above copyright 
+    //    notice, this list of conditions and the following disclaimers.
+    // 2. Redistributions in binary form must reproduce the above copyright 
+    //    notice in the documentation and/or other materials provided with 
+    //    the distribution.
+    // 3. The name of W. Randolph Franklin may not be used to endorse or 
+    //    promote products derived from this Software without specific 
+    //    prior written permission. 
+    //
+    // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED 
+    // TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL 
+    // THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF 
+    // CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS 
+    // IN THE SOFTWARE. 
+    boolean contains(double latitude, double longitude) {
+      // crossings algorithm is an odd-even algorithm, so we descend the tree xor'ing results along our path
+      boolean res = false;
+      if (latitude <= max) {
+        if (lat1 > latitude != lat2 > latitude) {
+          if (longitude < (lon1 - lon2) * (latitude - lat2) / (lat1 - lat2) + lon2) {
+            res = true;
+          }
+        }
+        if (left != null) {
+          res ^= left.contains(latitude, longitude);
+        }
+        if (right != null && latitude >= low) {
+          res ^= right.contains(latitude, longitude);
+        }
+      }
+      return res;
+    }
+    
+    /** Returns true if the box crosses any edge in this edge subtree */
+    boolean crosses(double minLat, double maxLat, double minLon, double maxLon) {
+      // we just have to cross one edge to answer the question, so we descend the tree and return when we do.
+      if (minLat <= max) {
+        // we compute line intersections of every polygon edge with every box line.
+        // if we find one, return true.
+        // for each box line (AB):
+        //   for each poly line (CD):
+        //     intersects = orient(C,D,A) * orient(C,D,B) <= 0 && orient(A,B,C) * orient(A,B,D) <= 0
+        double cy = lat1;
+        double dy = lat2;
+        double cx = lon1;
+        double dx = lon2;
+        
+        // optimization: see if the rectangle is outside of the "bounding box" of the polyline at all
+        // if not, don't waste our time trying more complicated stuff
+        boolean outside = (cy < minLat && dy < minLat) ||
+                          (cy > maxLat && dy > maxLat) ||
+                          (cx < minLon && dx < minLon) ||
+                          (cx > maxLon && dx > maxLon);
+        if (outside == false) {
+          // does box's top edge intersect polyline?
+          // ax = minLon, bx = maxLon, ay = maxLat, by = maxLat
+          if (orient(cx, cy, dx, dy, minLon, maxLat) * orient(cx, cy, dx, dy, maxLon, maxLat) <= 0 &&
+              orient(minLon, maxLat, maxLon, maxLat, cx, cy) * orient(minLon, maxLat, maxLon, maxLat, dx, dy) <= 0) {
+            return true;
+          }
+
+          // does box's right edge intersect polyline?
+          // ax = maxLon, bx = maxLon, ay = maxLat, by = minLat
+          if (orient(cx, cy, dx, dy, maxLon, maxLat) * orient(cx, cy, dx, dy, maxLon, minLat) <= 0 &&
+              orient(maxLon, maxLat, maxLon, minLat, cx, cy) * orient(maxLon, maxLat, maxLon, minLat, dx, dy) <= 0) {
+            return true;
+          }
+
+          // does box's bottom edge intersect polyline?
+          // ax = maxLon, bx = minLon, ay = minLat, by = minLat
+          if (orient(cx, cy, dx, dy, maxLon, minLat) * orient(cx, cy, dx, dy, minLon, minLat) <= 0 &&
+              orient(maxLon, minLat, minLon, minLat, cx, cy) * orient(maxLon, minLat, minLon, minLat, dx, dy) <= 0) {
+            return true;
+          }
+
+          // does box's left edge intersect polyline?
+          // ax = minLon, bx = minLon, ay = minLat, by = maxLat
+          if (orient(cx, cy, dx, dy, minLon, minLat) * orient(cx, cy, dx, dy, minLon, maxLat) <= 0 &&
+              orient(minLon, minLat, minLon, maxLat, cx, cy) * orient(minLon, minLat, minLon, maxLat, dx, dy) <= 0) {
+            return true;
+          }
+        }
+        
+        if (left != null) {
+          if (left.crosses(minLat, maxLat, minLon, maxLon)) {
+            return true;
+          }
+        }
+        
+        if (right != null && maxLat >= low) {
+          if (right.crosses(minLat, maxLat, minLon, maxLon)) {
+            return true;
+          }
+        }
+      }
+      return false;
+    }
+  }
+
+  /** 
+   * Creates an edge interval tree from a set of polygon vertices.
+   * @return root node of the tree.
+   */
+  private static Edge createTree(double polyLats[], double polyLons[]) {
+    // edge order is deterministic and reproducible based on the double values.
+    // TODO: make a real balanced tree instead :)
+    List<Integer> list = new ArrayList<Integer>(polyLats.length - 1);
+    for (int i = 1; i < polyLats.length; i++) {
+      list.add(i);
+    }
+    Collections.shuffle(list, new Random(Arrays.hashCode(polyLats) ^ Arrays.hashCode(polyLons)));
+    Edge root = null;
+    for (int i : list) {
+      double lat1 = polyLats[i-1];
+      double lon1 = polyLons[i-1];
+      double lat2 = polyLats[i];
+      double lon2 = polyLons[i];
+      Edge newNode = new Edge(lat1, lon1, lat2, lon2, Math.min(lat1, lat2), Math.max(lat1, lat2));
+      if (root == null) {
+        // add first node
+        root = newNode;
+      } else {
+        // traverse tree to find home for new node, along the path updating all parent's max value along the way.
+        Edge node = root;
+        while (true) {
+          node.max = Math.max(node.max, newNode.max);
+          if (newNode.low < node.low) {
+            if (node.left == null) {
+              node.left = newNode;
+              break;
+            }
+            node = node.left;
+          } else {
+            if (node.right == null) {
+              node.right = newNode;
+              break;
+            }
+            node = node.right;
+          }
+        }
+      }
+    }
+    return root;
+  }
+
+  /**
+   * Returns a positive value if points a, b, and c are arranged in counter-clockwise order,
+   * negative value if clockwise, zero if collinear.
+   */
+  // see the "Orient2D" method described here:
+  // http://www.cs.berkeley.edu/~jrs/meshpapers/robnotes.pdf
+  // https://www.cs.cmu.edu/~quake/robust.html
+  // Note that this one does not yet have the floating point tricks to be exact!
+  private static int orient(double ax, double ay, double bx, double by, double cx, double cy) {
+    double v1 = (bx - ax) * (cy - ay);
+    double v2 = (cx - ax) * (by - ay);
+    if (v1 > v2) {
+      return 1;
+    } else if (v1 < v2) {
+      return -1;
+    } else {
+      return 0;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fd5d880/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonTree.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonTree.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonTree.java
new file mode 100644
index 0000000..c939026
--- /dev/null
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonTree.java
@@ -0,0 +1,53 @@
+/*
+ * 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.geo.GeoTestUtil;
+import org.apache.lucene.geo.Polygon;
+import org.apache.lucene.geo.Rectangle;
+import org.apache.lucene.index.PointValues.Relation;
+import org.apache.lucene.util.LuceneTestCase;
+
+/** Test LatLonTree against the slower implementation for now */
+public class TestLatLonTree extends LuceneTestCase {
+  
+  /** test that contains() works the same as brute force */
+  public void testContainsRandom() {
+    for (int i = 0; i < 1000; i++) {
+      Polygon polygon = GeoTestUtil.nextPolygon();
+      LatLonTree tree = new LatLonTree(polygon);
+      for (int j = 0; j < 1000; j++) {
+        double point[] = GeoTestUtil.nextPointNear(polygon);
+        boolean expected = polygon.contains(point[0], point[1]);
+        assertEquals(expected, tree.contains(point[0], point[1]));
+      }
+    }
+  }
+  
+  /** test that relate() works the same as brute force */
+  public void testRelateRandom() {
+    for (int i = 0; i < 1000; i++) {
+      Polygon polygon = GeoTestUtil.nextPolygon();
+      LatLonTree tree = new LatLonTree(polygon);
+      for (int j = 0; j < 1000; j++) {
+        Rectangle box = GeoTestUtil.nextBoxNear(polygon);
+        Relation expected = polygon.relate(box.minLat, box.maxLat, box.minLon, box.maxLon);
+        assertEquals(expected, tree.relate(box.minLat, box.maxLat, box.minLon, box.maxLon));
+      }
+    }
+  }
+}


[23/50] [abbrv] lucene-solr:solr-5750: LUCENE-7240: Remove DocValues from LatLonPoint, add DocValuesField for that

Posted by ds...@apache.org.
LUCENE-7240: Remove DocValues from LatLonPoint, add DocValuesField for that


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

Branch: refs/heads/solr-5750
Commit: 45c48da54ad2bf09fd4c9559ba1c776ad9460d82
Parents: 7acf8ba
Author: Robert Muir <rm...@apache.org>
Authored: Sun Apr 24 17:15:30 2016 -0400
Committer: Robert Muir <rm...@apache.org>
Committed: Sun Apr 24 17:15:30 2016 -0400

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   4 +-
 .../lucene/document/LatLonDocValuesField.java   | 135 +++++++++++++++++++
 .../org/apache/lucene/document/LatLonPoint.java |  65 +++------
 .../document/LatLonPointDistanceComparator.java |   2 +-
 .../document/TestLatLonDocValuesField.java      |  30 +++++
 .../apache/lucene/document/TestLatLonPoint.java |   3 -
 .../document/TestLatLonPointDistanceSort.java   |  20 +--
 .../org/apache/lucene/document/TestNearest.java |   3 +-
 8 files changed, 197 insertions(+), 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/45c48da5/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 848e022..dc4cfcb 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -10,8 +10,8 @@ http://s.apache.org/luceneversions
 
 New Features
 
-* LUCENE-7099: Add LatLonPoint.newDistanceSort to the sandbox's
-  LatLonPoint. (Robert Muir)
+* LUCENE-7099: Add LatLonDocValuesField.newDistanceSort to the sandbox.
+  (Robert Muir)
 
 * LUCENE-7140: Add PlanetModel.bisection to spatial3d (Karl Wright via
   Mike McCandless)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/45c48da5/lucene/sandbox/src/java/org/apache/lucene/document/LatLonDocValuesField.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonDocValuesField.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonDocValuesField.java
new file mode 100644
index 0000000..20154d2
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonDocValuesField.java
@@ -0,0 +1,135 @@
+/*
+ * 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 static org.apache.lucene.geo.GeoEncodingUtils.decodeLatitude;
+import static org.apache.lucene.geo.GeoEncodingUtils.decodeLongitude;
+import static org.apache.lucene.geo.GeoEncodingUtils.encodeLatitude;
+import static org.apache.lucene.geo.GeoEncodingUtils.encodeLongitude;
+
+import org.apache.lucene.index.DocValuesType;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.search.FieldDoc;
+import org.apache.lucene.search.SortField;
+
+/** 
+ * An per-document location field.
+ * <p>
+ * Sorting by distance is efficient. Multiple values for the same field in one document
+ * is allowed. 
+ * <p>
+ * This field defines static factory methods for common operations:
+ * <ul>
+ *   <li>{@link #newDistanceSort newDistanceSort()} for ordering documents by distance from a specified location. 
+ * </ul>
+ * <p>
+ * If you also need query operations, you should add a separate {@link LatLonPoint} instance.
+ * If you also need to store the value, you should add a separate {@link StoredField} instance.
+ * <p>
+ * <b>WARNING</b>: Values are indexed with some loss of precision from the
+ * original {@code double} values (4.190951585769653E-8 for the latitude component
+ * and 8.381903171539307E-8 for longitude).
+ * @see LatLonPoint
+ */
+public class LatLonDocValuesField extends Field {
+
+  /**
+   * Type for a LatLonDocValuesField
+   * <p>
+   * Each value stores a 64-bit long where the upper 32 bits are the encoded latitude,
+   * and the lower 32 bits are the encoded longitude.
+   * @see org.apache.lucene.geo.GeoEncodingUtils#decodeLatitude(int)
+   * @see org.apache.lucene.geo.GeoEncodingUtils#decodeLongitude(int)
+   */
+  public static final FieldType TYPE = new FieldType();
+  static {
+    TYPE.setDocValuesType(DocValuesType.SORTED_NUMERIC);
+    TYPE.freeze();
+  }
+  
+  /** 
+   * Creates a new LatLonDocValuesField with the specified latitude and longitude
+   * @param name field name
+   * @param latitude latitude value: must be within standard +/-90 coordinate bounds.
+   * @param longitude longitude value: must be within standard +/-180 coordinate bounds.
+   * @throws IllegalArgumentException if the field name is null or latitude or longitude are out of bounds
+   */
+  public LatLonDocValuesField(String name, double latitude, double longitude) {
+    super(name, TYPE);
+    setLocationValue(latitude, longitude);
+  }
+  
+  /**
+   * Change the values of this field
+   * @param latitude latitude value: must be within standard +/-90 coordinate bounds.
+   * @param longitude longitude value: must be within standard +/-180 coordinate bounds.
+   * @throws IllegalArgumentException if latitude or longitude are out of bounds
+   */
+  public void setLocationValue(double latitude, double longitude) {
+    int latitudeEncoded = encodeLatitude(latitude);
+    int longitudeEncoded = encodeLongitude(longitude);
+    fieldsData = Long.valueOf((((long)latitudeEncoded) << 32) | (longitudeEncoded & 0xFFFFFFFFL));
+  }
+
+  /** helper: checks a fieldinfo and throws exception if its definitely not a LatLonDocValuesField */
+  static void checkCompatible(FieldInfo fieldInfo) {
+    // dv properties could be "unset", if you e.g. used only StoredField with this same name in the segment.
+    if (fieldInfo.getDocValuesType() != DocValuesType.NONE && fieldInfo.getDocValuesType() != TYPE.docValuesType()) {
+      throw new IllegalArgumentException("field=\"" + fieldInfo.name + "\" was indexed with docValuesType=" + fieldInfo.getDocValuesType() + 
+                                         " but this type has docValuesType=" + TYPE.docValuesType() + 
+                                         ", is the field really a LatLonDocValuesField?");
+    }
+  }
+  
+  @Override
+  public String toString() {
+    StringBuilder result = new StringBuilder();
+    result.append(getClass().getSimpleName());
+    result.append(" <");
+    result.append(name);
+    result.append(':');
+
+    long currentValue = Long.valueOf((Long)fieldsData);
+    result.append(decodeLatitude((int)(currentValue >> 32)));
+    result.append(',');
+    result.append(decodeLongitude((int)(currentValue & 0xFFFFFFFF)));
+
+    result.append('>');
+    return result.toString();
+  }
+
+  /**
+   * Creates a SortField for sorting by distance from a location.
+   * <p>
+   * This sort orders documents by ascending distance from the location. The value returned in {@link FieldDoc} for
+   * the hits contains a Double instance with the distance in meters.
+   * <p>
+   * If a document is missing the field, then by default it is treated as having {@link Double#POSITIVE_INFINITY} distance
+   * (missing values sort last).
+   * <p>
+   * If a document contains multiple values for the field, the <i>closest</i> distance to the location is used.
+   * 
+   * @param field field name. must not be null.
+   * @param latitude latitude at the center: must be within standard +/-90 coordinate bounds.
+   * @param longitude longitude at the center: must be within standard +/-180 coordinate bounds.
+   * @return SortField ordering documents by distance
+   * @throws IllegalArgumentException if {@code field} is null or location has invalid coordinates.
+   */
+  public static SortField newDistanceSort(String field, double latitude, double longitude) {
+    return new LatLonPointSortField(field, latitude, longitude);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/45c48da5/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java
index 0f6afe9..426a702 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java
@@ -24,7 +24,6 @@ import org.apache.lucene.codecs.lucene60.Lucene60PointsFormat;
 import org.apache.lucene.codecs.lucene60.Lucene60PointsReader;
 import org.apache.lucene.geo.GeoUtils;
 import org.apache.lucene.geo.Polygon;
-import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.PointValues;
@@ -38,7 +37,6 @@ import org.apache.lucene.search.MatchNoDocsQuery;
 import org.apache.lucene.search.PointRangeQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.ScoreDoc;
-import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.TopFieldDocs;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -63,21 +61,23 @@ import static org.apache.lucene.geo.GeoEncodingUtils.encodeLongitudeCeil;
  * <ul>
  *   <li>{@link #newBoxQuery newBoxQuery()} for matching points within a bounding box.
  *   <li>{@link #newDistanceQuery newDistanceQuery()} for matching points within a specified distance.
- *   <li>{@link #newDistanceSort newDistanceSort()} for ordering documents by distance from a specified location. 
  *   <li>{@link #newPolygonQuery newPolygonQuery()} for matching points within an arbitrary polygon.
  *   <li>{@link #nearest nearest()} for finding the k-nearest neighbors by distance.
  * </ul>
  * <p>
+ * If you also need per-document operations such as sort by distance, add a separate {@link LatLonDocValuesField} instance.
+ * If you also need to store the value, you should add a separate {@link StoredField} instance.
+ * <p>
  * <b>WARNING</b>: Values are indexed with some loss of precision from the
  * original {@code double} values (4.190951585769653E-8 for the latitude component
  * and 8.381903171539307E-8 for longitude).
  * @see PointValues
+ * @see LatLonDocValuesField
  */
 // TODO ^^^ that is very sandy and hurts the API, usage, and tests tremendously, because what the user passes
 // to the field is not actually what gets indexed. Float would be 1E-5 error vs 1E-7, but it might be
 // a better tradeoff? then it would be completely transparent to the user and lucene would be "lossless".
 public class LatLonPoint extends Field {
-  private long currentValue;
 
   /**
    * Type for an indexed LatLonPoint
@@ -87,7 +87,6 @@ public class LatLonPoint extends Field {
   public static final FieldType TYPE = new FieldType();
   static {
     TYPE.setDimensions(2, Integer.BYTES);
-    TYPE.setDocValuesType(DocValuesType.SORTED_NUMERIC);
     TYPE.freeze();
   }
   
@@ -98,13 +97,19 @@ public class LatLonPoint extends Field {
    * @throws IllegalArgumentException if latitude or longitude are out of bounds
    */
   public void setLocationValue(double latitude, double longitude) {
-    byte[] bytes = new byte[8];
+    final byte[] bytes;
+
+    if (fieldsData == null) {
+      bytes = new byte[8];
+      fieldsData = new BytesRef(bytes);
+    } else {
+      bytes = ((BytesRef) fieldsData).bytes;
+    }
+
     int latitudeEncoded = encodeLatitude(latitude);
     int longitudeEncoded = encodeLongitude(longitude);
     NumericUtils.intToSortableBytes(latitudeEncoded, bytes, 0);
     NumericUtils.intToSortableBytes(longitudeEncoded, bytes, Integer.BYTES);
-    fieldsData = new BytesRef(bytes);
-    currentValue = (((long)latitudeEncoded) << 32) | (longitudeEncoded & 0xFFFFFFFFL);
   }
 
   /** 
@@ -127,24 +132,14 @@ public class LatLonPoint extends Field {
     result.append(name);
     result.append(':');
 
-    result.append(decodeLatitude((int)(currentValue >> 32)));
+    byte bytes[] = ((BytesRef) fieldsData).bytes;
+    result.append(decodeLatitude(bytes, 0));
     result.append(',');
-    result.append(decodeLongitude((int)(currentValue & 0xFFFFFFFF)));
+    result.append(decodeLongitude(bytes, Integer.BYTES));
 
     result.append('>');
     return result.toString();
   }
-
-  /**
-   * Returns a 64-bit long, where the upper 32 bits are the encoded latitude,
-   * and the lower 32 bits are the encoded longitude.
-   * @see org.apache.lucene.geo.GeoEncodingUtils#decodeLatitude(int)
-   * @see org.apache.lucene.geo.GeoEncodingUtils#decodeLongitude(int)
-   */
-  @Override
-  public Number numericValue() {
-    return currentValue;
-  }
   
   /** sugar encodes a single point as a byte array */
   private static byte[] encode(double latitude, double longitude) {
@@ -175,11 +170,6 @@ public class LatLonPoint extends Field {
                                          " but this point type has bytesPerDim=" + TYPE.pointNumBytes() + 
                                          ", is the field really a LatLonPoint?");
     }
-    if (fieldInfo.getDocValuesType() != DocValuesType.NONE && fieldInfo.getDocValuesType() != TYPE.docValuesType()) {
-      throw new IllegalArgumentException("field=\"" + fieldInfo.name + "\" was indexed with docValuesType=" + fieldInfo.getDocValuesType() + 
-                                         " but this point type has docValuesType=" + TYPE.docValuesType() + 
-                                         ", is the field really a LatLonPoint?");
-    }
   }
 
   // static methods for generating queries
@@ -279,30 +269,9 @@ public class LatLonPoint extends Field {
   }
 
   /**
-   * Creates a SortField for sorting by distance from a location.
-   * <p>
-   * This sort orders documents by ascending distance from the location. The value returned in {@link FieldDoc} for
-   * the hits contains a Double instance with the distance in meters.
-   * <p>
-   * If a document is missing the field, then by default it is treated as having {@link Double#POSITIVE_INFINITY} distance
-   * (missing values sort last).
-   * <p>
-   * If a document contains multiple values for the field, the <i>closest</i> distance to the location is used.
-   * 
-   * @param field field name. must not be null.
-   * @param latitude latitude at the center: must be within standard +/-90 coordinate bounds.
-   * @param longitude longitude at the center: must be within standard +/-180 coordinate bounds.
-   * @return SortField ordering documents by distance
-   * @throws IllegalArgumentException if {@code field} is null or location has invalid coordinates.
-   */
-  public static SortField newDistanceSort(String field, double latitude, double longitude) {
-    return new LatLonPointSortField(field, latitude, longitude);
-  }
-
-  /**
    * Finds the {@code n} nearest indexed points to the provided point, according to Haversine distance.
    * <p>
-   * This is functionally equivalent to running {@link MatchAllDocsQuery} with a {@link #newDistanceSort},
+   * This is functionally equivalent to running {@link MatchAllDocsQuery} with a {@link LatLonDocValuesField#newDistanceSort},
    * but is far more efficient since it takes advantage of properties the indexed BKD tree.  Currently this
    * only works with {@link Lucene60PointsFormat} (used by the default codec).  Multi-valued fields are
    * currently not de-duplicated, so if a document had multiple instances of the specified field that

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/45c48da5/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointDistanceComparator.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointDistanceComparator.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointDistanceComparator.java
index 0306489..0b1d0c7 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointDistanceComparator.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointDistanceComparator.java
@@ -158,7 +158,7 @@ class LatLonPointDistanceComparator extends FieldComparator<Double> implements L
     LeafReader reader = context.reader();
     FieldInfo info = reader.getFieldInfos().fieldInfo(field);
     if (info != null) {
-      LatLonPoint.checkCompatible(info);
+      LatLonDocValuesField.checkCompatible(info);
     }
     currentDocs = DocValues.getSortedNumeric(reader, field);
     return this;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/45c48da5/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonDocValuesField.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonDocValuesField.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonDocValuesField.java
new file mode 100644
index 0000000..df934d1
--- /dev/null
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonDocValuesField.java
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+/** Simple tests for LatLonDocValuesField */
+public class TestLatLonDocValuesField extends LuceneTestCase {
+  public void testToString() throws Exception {
+    // looks crazy due to lossiness
+    assertEquals("LatLonDocValuesField <field:18.313693958334625,-65.22744401358068>",(new LatLonDocValuesField("field", 18.313694, -65.227444)).toString());
+    
+    // sort field
+    assertEquals("<distance:\"field\" latitude=18.0 longitude=19.0>", LatLonDocValuesField.newDistanceSort("field", 18.0, 19.0).toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/45c48da5/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPoint.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPoint.java
index acc9186..700eb56 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPoint.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPoint.java
@@ -33,8 +33,5 @@ public class TestLatLonPoint extends LuceneTestCase {
     
     // distance query does not quantize inputs
     assertEquals("field:18.0,19.0 +/- 25.0 meters", LatLonPoint.newDistanceQuery("field", 18, 19, 25).toString());
-    
-    // sort field
-    assertEquals("<distance:\"field\" latitude=18.0 longitude=19.0>", LatLonPoint.newDistanceSort("field", 18.0, 19.0).toString());
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/45c48da5/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPointDistanceSort.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPointDistanceSort.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPointDistanceSort.java
index e40f33d..6d825d2 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPointDistanceSort.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPointDistanceSort.java
@@ -40,7 +40,7 @@ import static org.apache.lucene.geo.GeoEncodingUtils.decodeLongitude;
 import static org.apache.lucene.geo.GeoEncodingUtils.encodeLatitude;
 import static org.apache.lucene.geo.GeoEncodingUtils.encodeLongitude;
 
-/** Simple tests for {@link LatLonPoint#newDistanceSort} */
+/** Simple tests for {@link LatLonDocValuesField#newDistanceSort} */
 public class TestLatLonPointDistanceSort extends LuceneTestCase {
 
   /** Add three points and sort by distance */
@@ -50,22 +50,22 @@ public class TestLatLonPointDistanceSort extends LuceneTestCase {
     
     // add some docs
     Document doc = new Document();
-    doc.add(new LatLonPoint("location", 40.759011, -73.9844722));
+    doc.add(new LatLonDocValuesField("location", 40.759011, -73.9844722));
     iw.addDocument(doc);
     
     doc = new Document();
-    doc.add(new LatLonPoint("location", 40.718266, -74.007819));
+    doc.add(new LatLonDocValuesField("location", 40.718266, -74.007819));
     iw.addDocument(doc);
     
     doc = new Document();
-    doc.add(new LatLonPoint("location", 40.7051157, -74.0088305));
+    doc.add(new LatLonDocValuesField("location", 40.7051157, -74.0088305));
     iw.addDocument(doc);
     
     IndexReader reader = iw.getReader();
     IndexSearcher searcher = newSearcher(reader);
     iw.close();
 
-    Sort sort = new Sort(LatLonPoint.newDistanceSort("location", 40.7143528, -74.0059731));
+    Sort sort = new Sort(LatLonDocValuesField.newDistanceSort("location", 40.7143528, -74.0059731));
     TopDocs td = searcher.search(new MatchAllDocsQuery(), 3, sort);
     
     FieldDoc d = (FieldDoc) td.scoreDocs[0];
@@ -91,18 +91,18 @@ public class TestLatLonPointDistanceSort extends LuceneTestCase {
     iw.addDocument(doc);
     
     doc = new Document();
-    doc.add(new LatLonPoint("location", 40.718266, -74.007819));
+    doc.add(new LatLonDocValuesField("location", 40.718266, -74.007819));
     iw.addDocument(doc);
     
     doc = new Document();
-    doc.add(new LatLonPoint("location", 40.7051157, -74.0088305));
+    doc.add(new LatLonDocValuesField("location", 40.7051157, -74.0088305));
     iw.addDocument(doc);
     
     IndexReader reader = iw.getReader();
     IndexSearcher searcher = newSearcher(reader);
     iw.close();
 
-    Sort sort = new Sort(LatLonPoint.newDistanceSort("location", 40.7143528, -74.0059731));
+    Sort sort = new Sort(LatLonDocValuesField.newDistanceSort("location", 40.7143528, -74.0059731));
     TopDocs td = searcher.search(new MatchAllDocsQuery(), 3, sort);
     
     FieldDoc d = (FieldDoc) td.scoreDocs[0];
@@ -199,7 +199,7 @@ public class TestLatLonPointDistanceSort extends LuceneTestCase {
         double lat = decodeLatitude(encodeLatitude(latRaw));
         double lon = decodeLongitude(encodeLongitude(lonRaw));
 
-        doc.add(new LatLonPoint("field", lat, lon));
+        doc.add(new LatLonDocValuesField("field", lat, lon));
         doc.add(new StoredField("lat", lat));
         doc.add(new StoredField("lon", lon));
       } // otherwise "missing"
@@ -234,7 +234,7 @@ public class TestLatLonPointDistanceSort extends LuceneTestCase {
       // randomize the topN a bit
       int topN = TestUtil.nextInt(random(), 1, reader.maxDoc());
       // sort by distance, then ID
-      SortField distanceSort = LatLonPoint.newDistanceSort("field", lat, lon);
+      SortField distanceSort = LatLonDocValuesField.newDistanceSort("field", lat, lon);
       distanceSort.setMissingValue(missingValue);
       Sort sort = new Sort(distanceSort, 
                            new SortField("id", SortField.Type.INT));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/45c48da5/lucene/sandbox/src/test/org/apache/lucene/document/TestNearest.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestNearest.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestNearest.java
index fc073c7..66630df 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestNearest.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestNearest.java
@@ -166,6 +166,7 @@ public class TestNearest extends LuceneTestCase {
       lons[id] = quantizeLon(GeoTestUtil.nextLongitude());
       Document doc = new Document();
       doc.add(new LatLonPoint("point", lats[id], lons[id]));
+      doc.add(new LatLonDocValuesField("point", lats[id], lons[id]));
       doc.add(new StoredField("id", id));
       w.addDocument(doc);
     }
@@ -216,7 +217,7 @@ public class TestNearest extends LuceneTestCase {
       }
 
       // Also test with MatchAllDocsQuery, sorting by distance:
-      TopFieldDocs fieldDocs = s.search(new MatchAllDocsQuery(), topN, new Sort(LatLonPoint.newDistanceSort("point", pointLat, pointLon)));
+      TopFieldDocs fieldDocs = s.search(new MatchAllDocsQuery(), topN, new Sort(LatLonDocValuesField.newDistanceSort("point", pointLat, pointLon)));
 
       ScoreDoc[] hits = LatLonPoint.nearest(s, "point", pointLat, pointLon, topN).scoreDocs;
       for(int i=0;i<topN;i++) {


[44/50] [abbrv] lucene-solr:solr-5750: SOLR-5750: revert ClusterStateMutator default state_format change; tests failed

Posted by ds...@apache.org.
SOLR-5750: revert ClusterStateMutator default state_format change; tests failed


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

Branch: refs/heads/solr-5750
Commit: 17c7a228c2c121b0775836562e1269a6385e04f9
Parents: 97b49d8
Author: David Smiley <ds...@apache.org>
Authored: Wed Apr 27 10:08:57 2016 -0400
Committer: David Smiley <ds...@apache.org>
Committed: Wed Apr 27 10:08:57 2016 -0400

----------------------------------------------------------------------
 .../java/org/apache/solr/cloud/overseer/ClusterStateMutator.java  | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/17c7a228/solr/core/src/java/org/apache/solr/cloud/overseer/ClusterStateMutator.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/ClusterStateMutator.java b/solr/core/src/java/org/apache/solr/cloud/overseer/ClusterStateMutator.java
index 2ace3cb..9db42ac 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/ClusterStateMutator.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/ClusterStateMutator.java
@@ -104,7 +104,8 @@ public class ClusterStateMutator {
       collectionProps.put("autoCreated", "true");
     }
 
-    String znode = message.getInt(DocCollection.STATE_FORMAT, 2) == 1 ? null
+    //TODO default to 2; but need to debug why BasicDistributedZk2Test fails early on
+    String znode = message.getInt(DocCollection.STATE_FORMAT, 1) == 1 ? null
         : ZkStateReader.getCollectionPath(cName);
 
     DocCollection newCollection = new DocCollection(cName,


[03/50] [abbrv] lucene-solr:solr-5750: LUCENE-7226: Add leniency support for filtering points, in order to be able to use OSM data.

Posted by ds...@apache.org.
LUCENE-7226: Add leniency support for filtering points, in order to be able to use OSM data.


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

Branch: refs/heads/solr-5750
Commit: 42b4f045c0dd831f383d6bd75e8fbfb9fdcb6290
Parents: 9394729
Author: Karl Wright <Da...@gmail.com>
Authored: Thu Apr 21 08:30:52 2016 -0400
Committer: Karl Wright <Da...@gmail.com>
Committed: Thu Apr 21 08:30:52 2016 -0400

----------------------------------------------------------------------
 .../spatial3d/geom/GeoPolygonFactory.java       | 129 ++++++++++++++++---
 .../lucene/spatial3d/geom/GeoPolygonTest.java   |   8 +-
 2 files changed, 118 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/42b4f045/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoPolygonFactory.java
----------------------------------------------------------------------
diff --git a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoPolygonFactory.java b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoPolygonFactory.java
index 61903b6..99fc7c9 100755
--- a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoPolygonFactory.java
+++ b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoPolygonFactory.java
@@ -48,7 +48,7 @@ public class GeoPolygonFactory {
     final List<GeoPoint> pointList) {
     return makeGeoPolygon(planetModel, pointList, null);
   }
-  
+
   /** Create a GeoPolygon using the specified points and holes, using order to determine 
    * siding of the polygon.  Much like ESRI, this method uses clockwise to indicate the space
    * on the same side of the shape as being inside, and counter-clockwise to indicate the
@@ -63,10 +63,32 @@ public class GeoPolygonFactory {
   public static GeoPolygon makeGeoPolygon(final PlanetModel planetModel,
     final List<GeoPoint> pointList,
     final List<GeoPolygon> holes) {
+    return makeGeoPolygon(planetModel, pointList, holes, 0.0);
+  }
+  
+  /** Create a GeoPolygon using the specified points and holes, using order to determine 
+   * siding of the polygon.  Much like ESRI, this method uses clockwise to indicate the space
+   * on the same side of the shape as being inside, and counter-clockwise to indicate the
+   * space on the opposite side as being inside.
+   * @param pointList is a list of the GeoPoints to build an arbitrary polygon out of.  If points go
+   *  clockwise from a given pole, then that pole should be within the polygon.  If points go
+   *  counter-clockwise, then that pole should be outside the polygon.
+   * @param holes is a list of polygons representing "holes" in the outside polygon.  Null == none.
+   * @param leniencyValue is the maximum distance (in units) that a point can be from the plane and still be considered as
+   *  belonging to the plane.  Any value greater than zero may cause some of the provided points that are in fact outside
+   *  the strict definition of co-planarity, but are within this distance, to be discarded for the purposes of creating a
+   *  "safe" polygon.
+   * @return a GeoPolygon corresponding to what was specified, or null if a valid polygon cannot be generated
+   *  from this input.
+   */
+  public static GeoPolygon makeGeoPolygon(final PlanetModel planetModel,
+    final List<GeoPoint> pointList,
+    final List<GeoPolygon> holes,
+    final double leniencyValue) {
     // First, exercise a sanity filter on the provided pointList, and remove identical points, linear points, and backtracks
     //System.err.println(" filtering "+pointList.size()+" points...");
     //final long startTime = System.currentTimeMillis();
-    final List<GeoPoint> filteredPointList = filterPoints(pointList);
+    final List<GeoPoint> filteredPointList = filterPoints(pointList, leniencyValue);
     //System.err.println("  ...done in "+(System.currentTimeMillis()-startTime)+"ms ("+((filteredPointList==null)?"degenerate":(filteredPointList.size()+" points"))+")");
     if (filteredPointList == null) {
       return null;
@@ -145,9 +167,10 @@ public class GeoPolygonFactory {
 
   /** Filter duplicate points and coplanar points.
    * @param input with input list of points
+   * @param leniencyValue is the allowed distance of a point from the plane for cleanup of overly detailed polygons
    * @return the filtered list, or null if we can't get a legit polygon from the input.
    */
-  static List<GeoPoint> filterPoints(final List<GeoPoint> input) {
+  static List<GeoPoint> filterPoints(final List<GeoPoint> input, final double leniencyValue) {
     
     final List<GeoPoint> noIdenticalPoints = new ArrayList<>(input.size());
     
@@ -200,7 +223,7 @@ public class GeoPolygonFactory {
     for  (int i = 0; i < noIdenticalPoints.size(); i++) {
       final SafePath startPath = new SafePath(null, noIdenticalPoints.get(i), i, null);
       // Search, with this as the start path.
-      final SafePath resultPath = findSafePath(startPath, noIdenticalPoints, getLegalIndex(i+1, noIdenticalPoints.size()), i);
+      final SafePath resultPath = findSafePath(startPath, noIdenticalPoints, getLegalIndex(i+1, noIdenticalPoints.size()), i, leniencyValue);
       if (resultPath != null && resultPath.previous != null) {
         // Read out result, maintaining ordering
         final List<GeoPoint> rval = new ArrayList<>(noIdenticalPoints.size());
@@ -220,10 +243,12 @@ public class GeoPolygonFactory {
    * @param pointIndex is the index of the point that represents the next possible point for consideration for path
    *  extension.
    * @param startPointIndex is index of the point that starts the current path, so that we can know when we are done.
+   * @param leniencyValue is the maximum allowed distance of a point being skipped from the revised polygon.  Pass zero if
+   *  no leniency desired.
    * @return null if there was no safe path found, or the safe path if one was discovered.
    */
   private static SafePath findSafePath(final SafePath currentPath, final List<GeoPoint> points, final int pointIndex,
-    final int startPointIndex) {
+    final int startPointIndex, final double leniencyValue) {
     //System.err.println("extending path...");
       
     // Loop across all possible path extensions, and consider each in turn
@@ -250,6 +275,24 @@ public class GeoPolygonFactory {
             } else if (considerPlane.evaluateIsZero(currentPath.previous.lastPoint)) {
               //System.err.println("  last point coplanar with this plane");
               isChoiceLegal = false;
+            } else {
+              // To guarantee that no planes we build are coplanar with edge points, we need to verify everything back from
+              // considerEndPoint back to the start of the path.  We build the edge from considerEndPoint back to each
+              // of the SafePath points already determined.  Then, we need to look at all triangles that include that edge and
+              // the SafePath points in between.  If all of those triangles are legal, we can be assured that adding the current
+              // proposed point is safe to do.
+              // This is, of course, a lot of work -- specifically, it's O(n^2) for each point in the path, which leads to an O(n^3)
+              // evaluation time overall!!
+              // The only alternative is to understand the cases under which these triangles would be introduced, and tailor the
+              // cleaning to catch those cases only.  Still need to figure that out.  The case that blows up is when *all* the points
+              // for a triangle are coplanar, so theoretically we don't even need to generate the triangle at all(!)
+              // 
+              // Build a plane that represents the third edge in this triangle, to guarantee that we can compose
+              // the polygon from triangles
+              final Plane thirdPlane = new Plane(currentPath.previous.lastPoint, considerEndPoint);
+              if (thirdPlane.evaluateIsZero(considerStartPoint)) {
+                isChoiceLegal = false;
+              }
             }
           }
         }
@@ -267,6 +310,13 @@ public class GeoPolygonFactory {
             } else if (considerPlane.evaluateIsZero(firstPlaneEndpoint.lastPoint)) {
               //System.err.println("  first point is coplanar with last plane");
               isChoiceLegal = false;
+            } else {
+              // Build a plane that represents the third edge in this triangle, to guarantee that we can compose
+              // the polygon from triangles
+              final Plane thirdPlane = new Plane(considerStartPoint, firstPlaneEndpoint.lastPoint);
+              if (thirdPlane.evaluateIsZero(considerEndPoint)) {
+                isChoiceLegal = false;
+              }
             }
           }
         }
@@ -275,7 +325,7 @@ public class GeoPolygonFactory {
           // All points between the start and end, if any, must be on the plane.
           int checkIndex = getLegalIndex(currentPath.lastPointIndex + 1, points.size());
           while (checkIndex != considerPointIndex) {
-            if (!considerPlane.evaluateIsZero(points.get(checkIndex))) {
+            if (Math.abs(considerPlane.evaluate(points.get(checkIndex))) >= Vector.MINIMUM_RESOLUTION + leniencyValue) {
               // This possibility is no good.  But does it say anything about other possibilities?  I think
               // it may mean we don't have to consider any further extensions.  I can't prove this, but
               // it makes this algorithm complete in not an insane period of time at least...
@@ -296,7 +346,7 @@ public class GeoPolygonFactory {
           }
           //System.err.println(" adding to path: "+considerEndPoint+"; "+considerPlane);
           final SafePath newPath = new SafePath(currentPath, considerEndPoint, considerPointIndex, considerPlane);
-          final SafePath result = findSafePath(newPath, points, nextPointIndex, startPointIndex);
+          final SafePath result = findSafePath(newPath, points, nextPointIndex, startPointIndex, leniencyValue);
           if (result != null) {
             return result;
           }
@@ -791,7 +841,9 @@ public class GeoPolygonFactory {
     // If there are less than three edges, something got messed up somehow.  Don't know how this
     // can happen but check.
     if (edgeBuffer.size() < 3) {
-      throw new IllegalStateException("Ending edge buffer had only "+edgeBuffer.size()+" edges");
+      // Linear...
+      // Here we can emit GeoWorld, but probably this means we had a broken poly to start with.
+      throw new IllegalArgumentException("Illegal polygon; polygon edges intersect each other");
     }
     
     // Create the list of points
@@ -970,11 +1022,12 @@ public class GeoPolygonFactory {
     }
 
     // Ok, figure out what we've accumulated.  If it is enough for a polygon, build it.
+      
     if (includedEdges.size() < 2) {
       //System.out.println("Done edge "+currentEdge+": no poly found");
       return false;
     }
-    
+
     // It's enough to build a convex polygon
     //System.out.println("Edge "+currentEdge+": Found complex poly");
     
@@ -990,21 +1043,45 @@ public class GeoPolygonFactory {
       // Degenerate case!!  There is no return edge -- or rather, we already have it.
       if (includedEdges.size() < 3) {
         // This means we found a degenerate cycle of edges.  If we emit a polygon at this point it
-        // has no contents, so we've clearly done something wrong, but not sure what.
-        throw new IllegalArgumentException("polygon was illegal (degenerate illegal two-edge cyclical polygon encountered in processing)");
+        // has no contents, so we generate no polygon.
+        return false;
       }
+      
+      // Now look for completely planar points.  This too is a degeneracy condition that we should
+      // return "false" for.
       Edge edge = firstEdge;
       points.add(edge.startPoint);
-      int i = 0;
+      int k = 0;
       while (true) {
         if (edge == lastEdge) {
           break;
         }
         points.add(edge.endPoint);
-        internalEdges.set(i++, edge.isInternal);
+        internalEdges.set(k++, edge.isInternal);
         edge = edgeBuffer.getNext(edge);
       }
       returnIsInternal = lastEdge.isInternal;
+      
+      // Look for coplanarity; abort if so
+      for (int i = 0; i < points.size(); i++) {
+        final GeoPoint start = points.get(i);
+        final GeoPoint end = points.get(getLegalIndex(i + 1, points.size()));
+        // We have to find the next point that is not on the plane between start and end.
+        // If there is no such point, it's an error.
+        final Plane planeToFind = new Plane(start, end);
+        int endPointIndex = -1;
+        for (int j = 0; j < points.size(); j++) {
+          final int index = getLegalIndex(j + i + 2, points.size());
+          if (!planeToFind.evaluateIsZero(points.get(index))) {
+            endPointIndex = index;
+            break;
+          }
+        }
+        if (endPointIndex == -1) {
+          return false;
+        }
+      }
+
       edgeBuffer.clear();
     } else {
       // Build the return edge (internal, of course)
@@ -1015,12 +1092,14 @@ public class GeoPolygonFactory {
       final List<Edge> edges = new ArrayList<Edge>(includedEdges.size());
       returnIsInternal = true;
 
+      // Now look for completely planar points.  This too is a degeneracy condition that we should
+      // return "false" for.
       Edge edge = firstEdge;
       points.add(edge.startPoint);
-      int i = 0;
+      int k = 0;
       while (true) {
         points.add(edge.endPoint);
-        internalEdges.set(i++, edge.isInternal);
+        internalEdges.set(k++, edge.isInternal);
         edges.add(edge);
         if (edge == lastEdge) {
           break;
@@ -1028,6 +1107,26 @@ public class GeoPolygonFactory {
         edge = edgeBuffer.getNext(edge);
       }
       
+      // Look for coplanarity; abort if so
+      for (int i = 0; i < points.size(); i++) {
+        final GeoPoint start = points.get(i);
+        final GeoPoint end = points.get(getLegalIndex(i + 1, points.size()));
+        // We have to find the next point that is not on the plane between start and end.
+        // If there is no such point, it's an error.
+        final Plane planeToFind = new Plane(start, end);
+        int endPointIndex = -1;
+        for (int j = 0; j < points.size(); j++) {
+          final int index = getLegalIndex(j + i + 2, points.size());
+          if (!planeToFind.evaluateIsZero(points.get(index))) {
+            endPointIndex = index;
+            break;
+          }
+        }
+        if (endPointIndex == -1) {
+          return false;
+        }
+      }
+
       // Modify the edge buffer
       edgeBuffer.replace(edges, returnEdge);
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/42b4f045/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/geom/GeoPolygonTest.java
----------------------------------------------------------------------
diff --git a/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/geom/GeoPolygonTest.java b/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/geom/GeoPolygonTest.java
index 2f71515..ce43c5b 100755
--- a/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/geom/GeoPolygonTest.java
+++ b/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/geom/GeoPolygonTest.java
@@ -44,7 +44,7 @@ public class GeoPolygonTest {
       originalPoints.add(point2);
       originalPoints.add(point2);
       originalPoints.add(point3);
-      final List<GeoPoint> filteredPoints =GeoPolygonFactory.filterPoints(originalPoints);
+      final List<GeoPoint> filteredPoints =GeoPolygonFactory.filterPoints(originalPoints, 0.0);
       assertEquals(3, filteredPoints.size());
       assertEquals(point1, filteredPoints.get(0));
       assertEquals(point2, filteredPoints.get(1));
@@ -57,7 +57,7 @@ public class GeoPolygonTest {
       originalPoints.add(point1);
       originalPoints.add(point3);
       originalPoints.add(point2);
-      final List<GeoPoint> filteredPoints =GeoPolygonFactory.filterPoints(originalPoints);
+      final List<GeoPoint> filteredPoints =GeoPolygonFactory.filterPoints(originalPoints, 0.0);
       assertEquals(3, filteredPoints.size());
       assertEquals(point2, filteredPoints.get(0));
       assertEquals(point1, filteredPoints.get(1));
@@ -71,7 +71,7 @@ public class GeoPolygonTest {
       originalPoints.add(point3);
       originalPoints.add(point4);
       originalPoints.add(point5);
-      final List<GeoPoint> filteredPoints =GeoPolygonFactory.filterPoints(originalPoints);
+      final List<GeoPoint> filteredPoints =GeoPolygonFactory.filterPoints(originalPoints, 0.0);
       assertEquals(3, filteredPoints.size());
       assertEquals(point1, filteredPoints.get(0));
       assertEquals(point3, filteredPoints.get(1));
@@ -85,7 +85,7 @@ public class GeoPolygonTest {
       originalPoints.add(point3);
       originalPoints.add(point4);
       System.err.println("Before: "+originalPoints);
-      final List<GeoPoint> filteredPoints =GeoPolygonFactory.filterPoints(originalPoints);
+      final List<GeoPoint> filteredPoints =GeoPolygonFactory.filterPoints(originalPoints, 0.0);
       System.err.println("After: "+filteredPoints);
       assertEquals(3, filteredPoints.size());
       assertEquals(point5, filteredPoints.get(0));


[37/50] [abbrv] lucene-solr:solr-5750: LUCENE-7252: add TestCoreParser.testTermQueryEmptyXML test

Posted by ds...@apache.org.
LUCENE-7252: add TestCoreParser.testTermQueryEmptyXML test


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

Branch: refs/heads/solr-5750
Commit: 9c69c4cf12f0cdbee6dd10f64f38cf32f7612670
Parents: 9166647
Author: Christine Poerschke <cp...@apache.org>
Authored: Tue Apr 26 17:39:33 2016 +0100
Committer: Christine Poerschke <cp...@apache.org>
Committed: Tue Apr 26 17:41:26 2016 +0100

----------------------------------------------------------------------
 .../lucene/queryparser/xml/TermQueryEmpty.xml    | 18 ++++++++++++++++++
 .../lucene/queryparser/xml/TestCoreParser.java   | 19 +++++++++++++++++++
 2 files changed, 37 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9c69c4cf/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/TermQueryEmpty.xml
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/TermQueryEmpty.xml b/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/TermQueryEmpty.xml
new file mode 100644
index 0000000..229a112
--- /dev/null
+++ b/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/TermQueryEmpty.xml
@@ -0,0 +1,18 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<TermQuery fieldName="contents"></TermQuery>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9c69c4cf/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/TestCoreParser.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/TestCoreParser.java b/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/TestCoreParser.java
index f252600..5f33545 100644
--- a/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/TestCoreParser.java
+++ b/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/TestCoreParser.java
@@ -66,6 +66,11 @@ public class TestCoreParser extends LuceneTestCase {
     dumpResults("TermQuery", q, 5);
   }
 
+  public void testTermQueryEmptyXML() throws ParserException, IOException {
+    parseShouldFail("TermQueryEmpty.xml",
+        "TermQuery has no text");
+  }
+
   public void testTermsQueryXML() throws ParserException, IOException {
     Query q = parse("TermsQuery.xml");
     dumpResults("TermsQuery", q, 5);
@@ -207,6 +212,20 @@ public class TestCoreParser extends LuceneTestCase {
     return indexData().searcher;
   }
 
+  protected void parseShouldFail(String xmlFileName, String expectedParserExceptionMessage) throws IOException {
+    Query q = null;
+    ParserException pe = null;
+    try {
+      q = parse(xmlFileName);
+    } catch (ParserException e) {
+      pe = e;
+    }
+    assertNull("for "+xmlFileName+" unexpectedly got "+q, q);
+    assertNotNull("expected a ParserException for "+xmlFileName, pe);
+    assertEquals("expected different ParserException for "+xmlFileName,
+        expectedParserExceptionMessage, pe.getMessage());
+  }
+
   protected Query parse(String xmlFileName) throws ParserException, IOException {
     try (InputStream xmlStream = TestCoreParser.class.getResourceAsStream(xmlFileName)) {
       assertNotNull("Test XML file " + xmlFileName + " cannot be found", xmlStream);


[06/50] [abbrv] lucene-solr:solr-5750: LUCENE-7238: Explicitly disable the query cache in MemoryIndex.createSearcher.

Posted by ds...@apache.org.
LUCENE-7238: Explicitly disable the query cache in MemoryIndex.createSearcher.


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

Branch: refs/heads/solr-5750
Commit: 2ee8426d5e53879424102afa9d6e59c1c62414c9
Parents: 81446cf
Author: Adrien Grand <jp...@gmail.com>
Authored: Thu Apr 21 17:01:43 2016 +0200
Committer: Adrien Grand <jp...@gmail.com>
Committed: Thu Apr 21 17:01:43 2016 +0200

----------------------------------------------------------------------
 lucene/CHANGES.txt                                                | 3 +++
 .../src/java/org/apache/lucene/index/memory/MemoryIndex.java      | 1 +
 2 files changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2ee8426d/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 7c30ef4..da5a345 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -73,6 +73,9 @@ Optimizations
 * LUCENE-7235: LRUQueryCache should not take a lock for segments that it will
   not cache on anyway. (Adrien Grand)
 
+* LUCENE-7238: Explicitly disable the query cache in MemoryIndex#createSearcher.
+  (Adrien Grand)
+
 Bug Fixes
 
 * LUCENE-7127: Fix corner case bugs in GeoPointDistanceQuery. (Robert Muir)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2ee8426d/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 58a1017..5b13301 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
@@ -666,6 +666,7 @@ public class MemoryIndex {
     MemoryIndexReader reader = new MemoryIndexReader();
     IndexSearcher searcher = new IndexSearcher(reader); // ensures no auto-close !!
     searcher.setSimilarity(normSimilarity);
+    searcher.setQueryCache(null);
     return searcher;
   }
 


[42/50] [abbrv] lucene-solr:solr-5750: LUCENE-7257: PointValues aggregated stats fail if the provided field does not have points on one of the leaves.

Posted by ds...@apache.org.
LUCENE-7257: PointValues aggregated stats fail if the provided field does not have points on one of the leaves.


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

Branch: refs/heads/solr-5750
Commit: c7cdf2832bbfbc46d9bfa3cb5041a821fbb7be1d
Parents: 9ce830d
Author: Adrien Grand <jp...@gmail.com>
Authored: Wed Apr 27 11:40:32 2016 +0200
Committer: Adrien Grand <jp...@gmail.com>
Committed: Wed Apr 27 11:41:04 2016 +0200

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  4 ++
 .../org/apache/lucene/index/PointValues.java    | 38 +++++++++------
 .../apache/lucene/index/TestPointValues.java    | 50 ++++++++++++++++++++
 3 files changed, 78 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c7cdf283/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index dc4cfcb..7a2cc03 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -104,6 +104,10 @@ Bug Fixes
 * LUCENE-7232: Fixed InetAddressPoint.newPrefixQuery, which was generating an
   incorrect query when the prefix length was not a multiple of 8. (Adrien Grand)
 
+* LUCENE-7257: Fixed PointValues#size(IndexReader, String), docCount,
+  minPackedValue and maxPackedValue to skip leaves that do not have points
+  rather than raising an IllegalStateException. (Adrien Grand)
+
 Documentation
 
 * LUCENE-7223: Improve XXXPoint javadocs to make it clear that you

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c7cdf283/lucene/core/src/java/org/apache/lucene/index/PointValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/PointValues.java b/lucene/core/src/java/org/apache/lucene/index/PointValues.java
index a4fd323..528f971 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PointValues.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PointValues.java
@@ -87,43 +87,51 @@ public abstract class PointValues {
   public static final int MAX_DIMENSIONS = BKDWriter.MAX_DIMS;
 
   /** Return the cumulated number of points across all leaves of the given
-   * {@link IndexReader}.
+   * {@link IndexReader}. Leaves that do not have points for the given field
+   * are ignored.
    *  @see PointValues#size(String) */
   public static long size(IndexReader reader, String field) throws IOException {
     long size = 0;
     for (LeafReaderContext ctx : reader.leaves()) {
-      PointValues values = ctx.reader().getPointValues();
-      if (values != null) {
-        size += values.size(field);
+      FieldInfo info = ctx.reader().getFieldInfos().fieldInfo(field);
+      if (info == null || info.getPointDimensionCount() == 0) {
+        continue;
       }
+      PointValues values = ctx.reader().getPointValues();
+      size += values.size(field);
     }
     return size;
   }
 
   /** Return the cumulated number of docs that have points across all leaves
-   * of the given {@link IndexReader}.
+   * of the given {@link IndexReader}. Leaves that do not have points for the
+   * given field are ignored.
    *  @see PointValues#getDocCount(String) */
   public static int getDocCount(IndexReader reader, String field) throws IOException {
     int count = 0;
     for (LeafReaderContext ctx : reader.leaves()) {
-      PointValues values = ctx.reader().getPointValues();
-      if (values != null) {
-        count += values.getDocCount(field);
+      FieldInfo info = ctx.reader().getFieldInfos().fieldInfo(field);
+      if (info == null || info.getPointDimensionCount() == 0) {
+        continue;
       }
+      PointValues values = ctx.reader().getPointValues();
+      count += values.getDocCount(field);
     }
     return count;
   }
 
   /** Return the minimum packed values across all leaves of the given
-   * {@link IndexReader}.
+   * {@link IndexReader}. Leaves that do not have points for the given field
+   * are ignored.
    *  @see PointValues#getMinPackedValue(String) */
   public static byte[] getMinPackedValue(IndexReader reader, String field) throws IOException {
     byte[] minValue = null;
     for (LeafReaderContext ctx : reader.leaves()) {
-      PointValues values = ctx.reader().getPointValues();
-      if (values == null) {
+      FieldInfo info = ctx.reader().getFieldInfos().fieldInfo(field);
+      if (info == null || info.getPointDimensionCount() == 0) {
         continue;
       }
+      PointValues values = ctx.reader().getPointValues();
       byte[] leafMinValue = values.getMinPackedValue(field);
       if (leafMinValue == null) {
         continue;
@@ -145,15 +153,17 @@ public abstract class PointValues {
   }
 
   /** Return the maximum packed values across all leaves of the given
-   * {@link IndexReader}.
+   * {@link IndexReader}. Leaves that do not have points for the given field
+   * are ignored.
    *  @see PointValues#getMaxPackedValue(String) */
   public static byte[] getMaxPackedValue(IndexReader reader, String field) throws IOException {
     byte[] maxValue = null;
     for (LeafReaderContext ctx : reader.leaves()) {
-      PointValues values = ctx.reader().getPointValues();
-      if (values == null) {
+      FieldInfo info = ctx.reader().getFieldInfos().fieldInfo(field);
+      if (info == null || info.getPointDimensionCount() == 0) {
         continue;
       }
+      PointValues values = ctx.reader().getPointValues();
       byte[] leafMaxValue = values.getMaxPackedValue(field);
       if (leafMaxValue == null) {
         continue;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c7cdf283/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 c7ca2dc..9f55ec3 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java
@@ -30,6 +30,7 @@ import org.apache.lucene.document.FloatPoint;
 import org.apache.lucene.document.IntPoint;
 import org.apache.lucene.document.LongPoint;
 import org.apache.lucene.document.StringField;
+import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.index.PointValues.IntersectVisitor;
 import org.apache.lucene.index.PointValues.Relation;
 import org.apache.lucene.index.PointValues;
@@ -652,6 +653,55 @@ public class TestPointValues extends LuceneTestCase {
     dir.close();
   }
 
+  public void testMergedStatsEmptyReader() throws IOException {
+    IndexReader reader = new MultiReader();
+    assertNull(PointValues.getMinPackedValue(reader, "field"));
+    assertNull(PointValues.getMaxPackedValue(reader, "field"));
+    assertEquals(0, PointValues.getDocCount(reader, "field"));
+    assertEquals(0, PointValues.size(reader, "field"));
+  }
+
+  public void testMergedStatsOneSegmentWithoutPoints() throws IOException {
+    Directory dir = new RAMDirectory();
+    IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null).setMergePolicy(NoMergePolicy.INSTANCE));
+    w.addDocument(new Document());
+    DirectoryReader.open(w).close();
+    Document doc = new Document();
+    doc.add(new IntPoint("field", Integer.MIN_VALUE));
+    w.addDocument(doc);
+    IndexReader reader = DirectoryReader.open(w);
+
+    assertArrayEquals(new byte[4], PointValues.getMinPackedValue(reader, "field"));
+    assertArrayEquals(new byte[4], PointValues.getMaxPackedValue(reader, "field"));
+    assertEquals(1, PointValues.getDocCount(reader, "field"));
+    assertEquals(1, PointValues.size(reader, "field"));
+
+    assertNull(PointValues.getMinPackedValue(reader, "field2"));
+    assertNull(PointValues.getMaxPackedValue(reader, "field2"));
+    assertEquals(0, PointValues.getDocCount(reader, "field2"));
+    assertEquals(0, PointValues.size(reader, "field2"));
+  }
+
+  public void testMergedStatsAllPointsDeleted() throws IOException {
+    Directory dir = new RAMDirectory();
+    IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null));
+    w.addDocument(new Document());
+    Document doc = new Document();
+    doc.add(new IntPoint("field", Integer.MIN_VALUE));
+    doc.add(new StringField("delete", "yes", Store.NO));
+    w.addDocument(doc);
+    w.forceMerge(1);
+    w.deleteDocuments(new Term("delete", "yes"));
+    w.addDocument(new Document());
+    w.forceMerge(1);
+    IndexReader reader = DirectoryReader.open(w);
+
+    assertNull(PointValues.getMinPackedValue(reader, "field"));
+    assertNull(PointValues.getMaxPackedValue(reader, "field"));
+    assertEquals(0, PointValues.getDocCount(reader, "field"));
+    assertEquals(0, PointValues.size(reader, "field"));
+  }
+
   public void testMergedStats() throws IOException {
     final int iters = atLeast(3);
     for (int iter = 0; iter < iters; ++iter) {


[41/50] [abbrv] lucene-solr:solr-5750: SOLR-5750: test asyncId usage.

Posted by ds...@apache.org.
SOLR-5750: test asyncId usage.


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

Branch: refs/heads/solr-5750
Commit: 97b49d8801aa74bfe3b57ebdbd60cfd301d76b33
Parents: 3ce00a5
Author: David Smiley <ds...@apache.org>
Authored: Tue Apr 26 20:19:46 2016 -0400
Committer: David Smiley <ds...@apache.org>
Committed: Tue Apr 26 20:19:46 2016 -0400

----------------------------------------------------------------------
 .../solr/cloud/TestCloudBackupRestore.java      | 64 ++++++++++----------
 1 file changed, 32 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/97b49d88/solr/core/src/test/org/apache/solr/cloud/TestCloudBackupRestore.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestCloudBackupRestore.java b/solr/core/src/test/org/apache/solr/cloud/TestCloudBackupRestore.java
index f0817e2..222c853 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestCloudBackupRestore.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestCloudBackupRestore.java
@@ -30,12 +30,11 @@ import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.response.RequestStatusState;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.ImplicitDocRouter;
 import org.apache.solr.common.cloud.Slice;
-import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.util.NamedList;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
@@ -88,7 +87,6 @@ public class TestCloudBackupRestore extends SolrCloudTestCase {
     }
 
     create.process(cluster.getSolrClient());
-    waitForCollection(collectionName);
     indexDocs(collectionName);
 
     if (!isImplicit && random().nextBoolean()) {
@@ -135,7 +133,6 @@ public class TestCloudBackupRestore extends SolrCloudTestCase {
 
   private void testBackupAndRestore(String collectionName) throws Exception {
     String backupName = "mytestbackup";
-    String restoreCollectionName = collectionName + "_restored";
 
     CloudSolrClient client = cluster.getSolrClient();
     DocCollection backupCollection = client.getZkStateReader().getClusterState().getCollection(collectionName);
@@ -147,30 +144,42 @@ public class TestCloudBackupRestore extends SolrCloudTestCase {
 
     log.info("Triggering Backup command");
 
-    CollectionAdminRequest.Backup backup = CollectionAdminRequest.backupCollection(collectionName, backupName)
-        .setLocation(location);
-    NamedList<Object> rsp = cluster.getSolrClient().request(backup);
-    assertEquals(0, ((NamedList)rsp.get("responseHeader")).get("status"));
+    {
+      CollectionAdminRequest.Backup backup = CollectionAdminRequest.backupCollection(collectionName, backupName)
+          .setLocation(location);
+      if (random().nextBoolean()) {
+        assertEquals(0, backup.process(client).getStatus());
+      } else {
+        assertEquals(RequestStatusState.COMPLETED, backup.processAndWait(client, 30));//async
+      }
+    }
 
     log.info("Triggering Restore command");
 
-    //nocommit test with async
-    CollectionAdminRequest.Restore restore = CollectionAdminRequest.restoreCollection(restoreCollectionName, backupName)
-        .setLocation(location);
-    if (origShardToDocCount.size() > cluster.getJettySolrRunners().size()) {
-      // may need to increase maxShardsPerNode (e.g. if it was shard split, then now we need more)
-      restore.getCreateOptions().setMaxShardsPerNode(origShardToDocCount.size());
-    }
-    Properties props = new Properties();
-    props.setProperty("customKey", "customVal");
-    restore.getCreateOptions().setProperties(props);
+    String restoreCollectionName = collectionName + "_restored";
     boolean sameConfig = random().nextBoolean();
-    if (sameConfig==false) {
-      restore.getCreateOptions().setConfigName("customConfigName");//nocommit ugh, this is deprecated
+
+    {
+      CollectionAdminRequest.Restore restore = CollectionAdminRequest.restoreCollection(restoreCollectionName, backupName)
+              .setLocation(location);
+      if (origShardToDocCount.size() > cluster.getJettySolrRunners().size()) {
+        // may need to increase maxShardsPerNode (e.g. if it was shard split, then now we need more)
+        restore.getCreateOptions().setMaxShardsPerNode(origShardToDocCount.size());
+      }
+      Properties props = new Properties();
+      props.setProperty("customKey", "customVal");
+      restore.getCreateOptions().setProperties(props);
+      if (sameConfig==false) {
+        restore.getCreateOptions().setConfigName("customConfigName");//nocommit ugh, this is deprecated
+      }
+      if (random().nextBoolean()) {
+        assertEquals(0, restore.process(client).getStatus());
+      } else {
+        assertEquals(RequestStatusState.COMPLETED, restore.processAndWait(client, 30));//async
+      }
+      AbstractDistribZkTestBase.waitForRecoveriesToFinish(
+          restoreCollectionName, cluster.getSolrClient().getZkStateReader(), log.isDebugEnabled(), true, 30);
     }
-    rsp = cluster.getSolrClient().request(restore); // DO IT!
-    assertEquals(0, ((NamedList)rsp.get("responseHeader")).get("status"));
-    waitForCollection(restoreCollectionName);
 
     //Check the number of results are the same
     DocCollection restoreCollection = client.getZkStateReader().getClusterState().getCollection(restoreCollectionName);
@@ -204,13 +213,4 @@ public class TestCloudBackupRestore extends SolrCloudTestCase {
     return shardToDocCount;
   }
 
-  public void waitForCollection(String collection) throws Exception {
-    // note: NUM_SHARDS may be too small because of shard split, but that's okay?
-    ZkStateReader zkStateReader = cluster.getSolrClient().getZkStateReader();
-    AbstractFullDistribZkTestBase.waitForCollection(zkStateReader, collection, NUM_SHARDS);
-    AbstractDistribZkTestBase.waitForRecoveriesToFinish(collection, zkStateReader, log.isDebugEnabled(), true, 30);
-    AbstractDistribZkTestBase.assertAllActive(collection, zkStateReader);
-    assertFalse(zkStateReader.getClusterState().getCollection(collection).getActiveSlices().isEmpty());
-  }
-
 }


[40/50] [abbrv] lucene-solr:solr-5750: SOLR-5750: waiting to finish recovery will also wait to be out of construction state.

Posted by ds...@apache.org.
SOLR-5750: waiting to finish recovery will also wait to be out of construction state.


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

Branch: refs/heads/solr-5750
Commit: 3ce00a557665535eb748cf328bba1ebaf51b7af2
Parents: e8d2968
Author: David Smiley <ds...@apache.org>
Authored: Tue Apr 26 20:18:56 2016 -0400
Committer: David Smiley <ds...@apache.org>
Committed: Tue Apr 26 20:18:56 2016 -0400

----------------------------------------------------------------------
 .../cloud/OverseerCollectionMessageHandler.java |  3 ++-
 .../org/apache/solr/common/cloud/Slice.java     |  7 ++++---
 .../solr/cloud/AbstractDistribZkTestBase.java   | 20 ++++++++++++++------
 3 files changed, 20 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ce00a55/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java b/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
index 1832ead..40907f5 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
@@ -414,6 +414,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
 
     //Mark all shards in CONSTRUCTION STATE while we restore the data
     {
+      //TODO might instead createCollection accept an initial state?  Is there a race?
       Map<String, Object> propMap = new HashMap<>();
       propMap.put(Overseer.QUEUE_OPERATION, OverseerAction.UPDATESHARDSTATE.toLower());
       for (Slice shard : restoreCollection.getSlices()) {
@@ -448,9 +449,9 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
     //Copy data from backed up index to each replica
     for (Slice slice: restoreCollection.getSlices()) {
       ModifiableSolrParams params = new ModifiableSolrParams();
+      params.set(CoreAdminParams.ACTION, CoreAdminAction.RESTORECORE.toString());
       params.set(NAME, "snapshot." + slice.getName());
       params.set("location", backupPath.toString());
-      params.set(CoreAdminParams.ACTION, CoreAdminAction.RESTORECORE.toString());
       sliceCmd(clusterState, params, null, slice, shardHandler, asyncId, requestMap);
     }
     processResponses(new NamedList(), shardHandler, true, "Could not restore core", asyncId, requestMap);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ce00a55/solr/solrj/src/java/org/apache/solr/common/cloud/Slice.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/Slice.java b/solr/solrj/src/java/org/apache/solr/common/cloud/Slice.java
index 41478fd..163561a 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/Slice.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/Slice.java
@@ -50,11 +50,11 @@ public class Slice extends ZkNodeProps {
   /** The slice's state. */
   public enum State {
     
-    /** The default state of a slice. */
+    /** The normal/default state of a shard. */
     ACTIVE,
     
     /**
-     * A slice is put in that state after it has been successfully split. See
+     * A shard is put in that state after it has been successfully split. See
      * <a href="https://cwiki.apache.org/confluence/display/solr/Collections+API#CollectionsAPI-api3">
      * the reference guide</a> for more details.
      */
@@ -62,7 +62,8 @@ public class Slice extends ZkNodeProps {
     
     /**
      * When a shard is split, the new sub-shards are put in that state while the
-     * split operation is in progress. A shard in that state still receives
+     * split operation is in progress. It's also used when the shard is undergoing data restoration.
+     * A shard in this state still receives
      * update requests from the parent shard leader, however does not participate
      * in distributed search.
      */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3ce00a55/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java
index 4ac6d5a..42724dc 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java
@@ -149,7 +149,12 @@ public abstract class AbstractDistribZkTestBase extends BaseDistributedSearchTes
       Map<String,Slice> slices = clusterState.getSlicesMap(collection);
       assertNotNull("Could not find collection:" + collection, slices);
       for (Map.Entry<String,Slice> entry : slices.entrySet()) {
-        Map<String,Replica> shards = entry.getValue().getReplicasMap();
+        Slice slice = entry.getValue();
+        if (slice.getState() == Slice.State.CONSTRUCTION) { // similar to replica recovering; pretend its the same thing
+          if (verbose) System.out.println("Found a slice in construction state; will wait.");
+          sawLiveRecovering = true;
+        }
+        Map<String,Replica> shards = slice.getReplicasMap();
         for (Map.Entry<String,Replica> shard : shards.entrySet()) {
           if (verbose) System.out.println("replica:" + shard.getValue().getName() + " rstate:"
               + shard.getValue().getStr(ZkStateReader.STATE_PROP)
@@ -244,12 +249,15 @@ public abstract class AbstractDistribZkTestBase extends BaseDistributedSearchTes
         throw new IllegalArgumentException("Cannot find collection:" + collection);
       }
       for (Map.Entry<String,Slice> entry : slices.entrySet()) {
-        Map<String,Replica> shards = entry.getValue().getReplicasMap();
+        Slice slice = entry.getValue();
+        if (slice.getState() != Slice.State.ACTIVE) {
+          fail("Not all shards are ACTIVE - found a shard " + slice.getName() + " that is: " + slice.getState());
+        }
+        Map<String,Replica> shards = slice.getReplicasMap();
         for (Map.Entry<String,Replica> shard : shards.entrySet()) {
-
-          final Replica.State state = shard.getValue().getState();
-          if (state != Replica.State.ACTIVE) {
-            fail("Not all shards are ACTIVE - found a shard that is: " + state.toString());
+          Replica replica = shard.getValue();
+          if (replica.getState() != Replica.State.ACTIVE) {
+            fail("Not all replicas are ACTIVE - found a replica " + replica.getName() + " that is: " + replica.getState());
           }
         }
       }


[13/50] [abbrv] lucene-solr:solr-5750: Merge remote-tracking branch 'origin/master'

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

Branch: refs/heads/solr-5750
Commit: 7b89d0f698554aef2e9016f23b8345e372d54848
Parents: 827c670 4fd5d88
Author: Noble Paul <no...@apache.org>
Authored: Fri Apr 22 12:34:03 2016 +0530
Committer: Noble Paul <no...@apache.org>
Committed: Fri Apr 22 12:34:03 2016 +0530

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   7 +-
 .../java/org/apache/lucene/util/BitUtil.java    |   4 +-
 .../replicator/http/ReplicationService.java     |   1 +
 .../org/apache/lucene/document/LatLonGrid.java  |  26 +-
 .../document/LatLonPointInPolygonQuery.java     |  61 +--
 .../org/apache/lucene/document/LatLonTree.java  | 401 +++++++++++++++++++
 .../apache/lucene/document/TestLatLonTree.java  |  53 +++
 .../geopoint/document/GeoPointField.java        | 110 ++++-
 .../geopoint/document/GeoPointTokenStream.java  |   4 +-
 .../geopoint/search/GeoPointInPolygonQuery.java |   1 -
 .../search/GeoPointNumericTermsEnum.java        |  17 +-
 .../search/GeoPointPrefixTermsEnum.java         |  26 +-
 .../GeoPointTermQueryConstantScoreWrapper.java  |   7 +-
 .../lucene/spatial/util/GeoEncodingUtils.java   | 146 -------
 .../lucene/spatial/util/MortonEncoder.java      | 103 +++++
 .../geopoint/search/TestGeoPointQuery.java      |   5 +-
 .../search/TestLegacyGeoPointQuery.java         |   5 +-
 .../spatial/util/TestGeoEncodingUtils.java      | 102 -----
 .../lucene/spatial/util/TestGeoPointField.java  |  51 +++
 .../lucene/spatial/util/TestMortonEncoder.java  | 108 +++++
 solr/CHANGES.txt                                |   2 +
 .../org/apache/solr/servlet/HttpSolrCall.java   |  11 +-
 .../apache/solr/servlet/LoadAdminUiServlet.java |   9 +-
 .../solr/servlet/ServletInputStreamWrapper.java | 105 +++++
 .../servlet/ServletOutputStreamWrapper.java     | 140 +++++++
 .../apache/solr/servlet/SolrDispatchFilter.java |  92 +++++
 .../apache/solr/servlet/SolrRequestParsers.java |  12 +-
 .../client/solrj/embedded/JettyWebappTest.java  |   4 +-
 28 files changed, 1240 insertions(+), 373 deletions(-)
----------------------------------------------------------------------



[07/50] [abbrv] lucene-solr:solr-5750: SOLR-8992: Restore Schema API GET method functionality removed in 6.0

Posted by ds...@apache.org.
SOLR-8992: Restore Schema API GET method functionality removed in 6.0


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

Branch: refs/heads/solr-5750
Commit: e8cc19eb885c46d25b56fdd681825712516050c9
Parents: 81446cf
Author: Noble Paul <no...@apache.org>
Authored: Thu Apr 21 20:38:20 2016 +0530
Committer: Noble Paul <no...@apache.org>
Committed: Thu Apr 21 20:38:20 2016 +0530

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   2 +
 .../org/apache/solr/handler/SchemaHandler.java  |   2 +-
 .../apache/solr/response/SchemaXmlWriter.java   |  27 ++-
 .../org/apache/solr/schema/IndexSchema.java     | 177 ++++++++++++++-----
 .../schema/TestCopyFieldCollectionResource.java |  34 ++++
 .../TestDynamicFieldCollectionResource.java     |  24 +++
 .../schema/TestFieldCollectionResource.java     |  33 ++++
 7 files changed, 239 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e8cc19eb/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 31ae74c..9c8d03b 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -148,6 +148,8 @@ Bug Fixes
 
 * SOLR-8971: Preserve root cause when wrapping exceptions (hossman)
 
+* SOLR-8992: Restore Schema API GET method functionality removed in 6.0 (noble, Steve Rowe)
+
 Optimizations
 ----------------------
 * SOLR-8722: Don't force a full ZkStateReader refresh on every Overseer operation.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e8cc19eb/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java b/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java
index da24c25..4da557d 100644
--- a/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/SchemaHandler.java
@@ -185,7 +185,7 @@ public class SchemaHandler extends RequestHandlerBase implements SolrCoreAware,
           if (parts.get(0).isEmpty()) parts.remove(0);
           if (parts.size() > 1 && level2.containsKey(parts.get(1))) {
             String realName = level2.get(parts.get(1));
-            SimpleOrderedMap<Object> propertyValues = req.getSchema().getNamedPropertyValues(req.getParams());
+            Map propertyValues = req.getSchema().getNamedPropertyValues(realName, req.getParams());
             Object o = propertyValues.get(realName);
             if(parts.size()> 2) {
               String name = parts.get(2);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e8cc19eb/solr/core/src/java/org/apache/solr/response/SchemaXmlWriter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/response/SchemaXmlWriter.java b/solr/core/src/java/org/apache/solr/response/SchemaXmlWriter.java
index 4ed4d8f..a38fe36 100644
--- a/solr/core/src/java/org/apache/solr/response/SchemaXmlWriter.java
+++ b/solr/core/src/java/org/apache/solr/response/SchemaXmlWriter.java
@@ -78,8 +78,8 @@ public class SchemaXmlWriter extends TextResponseWriter {
       writer.write(MANAGED_SCHEMA_DO_NOT_EDIT_WARNING);
     }
 
-    @SuppressWarnings("unchecked") SimpleOrderedMap<Object> schemaProperties
-        = (SimpleOrderedMap<Object>)rsp.getValues().get(IndexSchema.SCHEMA);
+    @SuppressWarnings("unchecked") Map<String,Object> schemaProperties
+        = (Map<String , Object>)rsp.getValues().get(IndexSchema.SCHEMA);
 
     openStartTag(IndexSchema.SCHEMA);
     writeAttr(IndexSchema.NAME, schemaProperties.get(IndexSchema.NAME).toString());
@@ -87,34 +87,33 @@ public class SchemaXmlWriter extends TextResponseWriter {
     closeStartTag(false);
     incLevel();
 
-    for (int schemaPropNum = 0 ; schemaPropNum < schemaProperties.size() ; ++schemaPropNum) {
-      String schemaPropName = schemaProperties.getName(schemaPropNum);
+    for (Map.Entry<String, Object> entry : schemaProperties.entrySet()) {
+      String schemaPropName = entry.getKey();
+      Object val = entry.getValue();
       if (schemaPropName.equals(IndexSchema.NAME) || schemaPropName.equals(IndexSchema.VERSION)) {
         continue;
       }
       if (schemaPropName.equals(IndexSchema.UNIQUE_KEY)) {
         openStartTag(IndexSchema.UNIQUE_KEY);
         closeStartTag(false);
-        writer.write(schemaProperties.getVal(schemaPropNum).toString());
+        writer.write(val.toString());
         endTag(IndexSchema.UNIQUE_KEY, false);
       } else if (schemaPropName.equals(IndexSchema.DEFAULT_SEARCH_FIELD)) {
         openStartTag(IndexSchema.DEFAULT_SEARCH_FIELD);
         closeStartTag(false);
-        writer.write(schemaProperties.getVal(schemaPropNum).toString());
+        writer.write(val.toString());
         endTag(IndexSchema.DEFAULT_SEARCH_FIELD, false);
       } else if (schemaPropName.equals(IndexSchema.SOLR_QUERY_PARSER)) {
         openStartTag(IndexSchema.SOLR_QUERY_PARSER);
-        @SuppressWarnings("unchecked") SimpleOrderedMap<Object> solrQueryParserProperties
-            = (SimpleOrderedMap<Object>)schemaProperties.getVal(schemaPropNum);
-        writeAttr(IndexSchema.DEFAULT_OPERATOR, solrQueryParserProperties.get(IndexSchema.DEFAULT_OPERATOR).toString());
+        writeAttr(IndexSchema.DEFAULT_OPERATOR, ((Map<String ,Object>) val).get(IndexSchema.DEFAULT_OPERATOR).toString());
         closeStartTag(true);
       } else if (schemaPropName.equals(IndexSchema.SIMILARITY)) {
-        writeSimilarity((SimpleOrderedMap<Object>) schemaProperties.getVal(schemaPropNum));
+        writeSimilarity((SimpleOrderedMap<Object>) val);
       } else if (schemaPropName.equals(IndexSchema.FIELD_TYPES)) {
-        writeFieldTypes((List<SimpleOrderedMap<Object>>)schemaProperties.getVal(schemaPropNum));
+        writeFieldTypes((List<SimpleOrderedMap<Object>>) val);
       } else if (schemaPropName.equals(IndexSchema.FIELDS)) {
         @SuppressWarnings("unchecked") List<SimpleOrderedMap<Object>> fieldPropertiesList
-            = (List<SimpleOrderedMap<Object>>)schemaProperties.getVal(schemaPropNum);
+            = (List<SimpleOrderedMap<Object>>) val;
         for (SimpleOrderedMap<Object> fieldProperties : fieldPropertiesList) {
           openStartTag(IndexSchema.FIELD);
           for (int fieldPropNum = 0 ; fieldPropNum < fieldProperties.size() ; ++fieldPropNum) {
@@ -124,7 +123,7 @@ public class SchemaXmlWriter extends TextResponseWriter {
         }
       } else if (schemaPropName.equals(IndexSchema.DYNAMIC_FIELDS)) {
         @SuppressWarnings("unchecked") List<SimpleOrderedMap<Object>> dynamicFieldPropertiesList 
-            = (List<SimpleOrderedMap<Object>>)schemaProperties.getVal(schemaPropNum);
+            = (List<SimpleOrderedMap<Object>>) val;
         for (SimpleOrderedMap<Object> dynamicFieldProperties : dynamicFieldPropertiesList) {
           openStartTag(IndexSchema.DYNAMIC_FIELD);
           for (int dynamicFieldPropNum = 0 ; dynamicFieldPropNum < dynamicFieldProperties.size() ; ++dynamicFieldPropNum) {
@@ -135,7 +134,7 @@ public class SchemaXmlWriter extends TextResponseWriter {
         }
       } else if (schemaPropName.equals(IndexSchema.COPY_FIELDS)) {
         @SuppressWarnings("unchecked") List<SimpleOrderedMap<Object>> copyFieldPropertiesList
-            = (List<SimpleOrderedMap<Object>>)schemaProperties.getVal(schemaPropNum);
+            = (List<SimpleOrderedMap<Object>>) val;
         for (SimpleOrderedMap<Object> copyFieldProperties : copyFieldPropertiesList) {
           openStartTag(IndexSchema.COPY_FIELD);
           for (int copyFieldPropNum = 0 ; copyFieldPropNum < copyFieldProperties.size() ; ++ copyFieldPropNum) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e8cc19eb/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/IndexSchema.java b/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
index 7f7bd73..f71db0a 100644
--- a/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
+++ b/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
@@ -20,11 +20,15 @@ import java.io.IOException;
 import java.io.Writer;
 import java.lang.invoke.MethodHandles;
 import java.util.*;
+import java.util.function.Function;
 import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 import javax.xml.xpath.XPath;
 import javax.xml.xpath.XPathConstants;
 import javax.xml.xpath.XPathExpressionException;
 
+import com.google.common.base.Functions;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.DelegatingAnalyzerWrapper;
 import org.apache.lucene.index.DocValuesType;
@@ -32,19 +36,21 @@ import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexableField;
-import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.uninverting.UninvertingReader;
 import org.apache.lucene.util.Version;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.MapSolrParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.Pair;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.core.Config;
+import org.apache.solr.core.MapSerializable;
 import org.apache.solr.core.SolrConfig;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.request.LocalSolrQueryRequest;
@@ -63,6 +69,7 @@ import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
 import org.xml.sax.InputSource;
 
+import static java.util.Arrays.asList;
 import static java.util.Collections.singletonList;
 import static java.util.Collections.singletonMap;
 
@@ -780,7 +787,7 @@ public class IndexSchema {
    * @param fields The sequence of {@link org.apache.solr.schema.SchemaField}
    */
   public void registerDynamicFields(SchemaField... fields) {
-    List<DynamicField> dynFields = new ArrayList<>(Arrays.asList(dynamicFields));
+    List<DynamicField> dynFields = new ArrayList<>(asList(dynamicFields));
     for (SchemaField field : fields) {
       if (isDuplicateDynField(dynFields, field)) {
         log.debug("dynamic field already exists: dynamic field: [" + field.getName() + "]");
@@ -1352,58 +1359,138 @@ public class IndexSchema {
   /**
    * Get a map of property name -&gt; value for the whole schema.
    */
-  public SimpleOrderedMap<Object> getNamedPropertyValues() {
-    return getNamedPropertyValues(new MapSolrParams(Collections.EMPTY_MAP));
+  public Map getNamedPropertyValues() {
+    return getNamedPropertyValues(null, new MapSolrParams(Collections.EMPTY_MAP));
+  }
+
+  static class SchemaProps implements MapSerializable {
+    private static final String SOURCE_FIELD_LIST = IndexSchema.SOURCE + "." + CommonParams.FL;
+    private static final String DESTINATION_FIELD_LIST = IndexSchema.DESTINATION + "." + CommonParams.FL;
+    private final String name;
+    private final SolrParams params;
+    private final IndexSchema schema;
+    boolean showDefaults, includeDynamic;
+    Set<String> requestedFields;
+    private Set<String> requestedSourceFields;
+    private Set<String> requestedDestinationFields;
+
+
+    enum Handler {
+      NAME(IndexSchema.NAME, sp -> sp.schema.getSchemaName()),
+      VERSION(IndexSchema.VERSION, sp -> sp.schema.getVersion()),
+      UNIQUE_KEY(IndexSchema.UNIQUE_KEY, sp -> sp.schema.uniqueKeyFieldName),
+      DEFAULT_SEARCH_FIELD(IndexSchema.DEFAULT_SEARCH_FIELD, sp -> sp.schema.defaultSearchFieldName),
+      SOLR_QUERY_PARSER(IndexSchema.SOLR_QUERY_PARSER, sp -> sp.schema.isExplicitQueryParserDefaultOperator ?
+          singletonMap(DEFAULT_OPERATOR, sp.schema.queryParserDefaultOperator) :
+          null),
+      SIMILARITY(IndexSchema.SIMILARITY, sp -> sp.schema.isExplicitSimilarity ?
+          sp.schema.similarityFactory.getNamedPropertyValues() :
+          null),
+      FIELD_TYPES(IndexSchema.FIELD_TYPES, sp -> new TreeMap<>(sp.schema.fieldTypes)
+          .values().stream()
+          .map(it -> it.getNamedPropertyValues(sp.showDefaults))
+          .collect(Collectors.toList())),
+
+      DYNAMIC_FIELDS(IndexSchema.DYNAMIC_FIELDS, sp -> Stream.of(sp.schema.dynamicFields)
+          .filter(it -> !it.getRegex().startsWith(INTERNAL_POLY_FIELD_PREFIX))
+          .filter(it -> sp.requestedFields == null || sp.requestedFields.contains(it.getPrototype().getName()))
+          .map(it -> sp.getProperties(it.getPrototype()))
+          .collect(Collectors.toList())),
+      FIELDS(IndexSchema.FIELDS, sp -> {
+        List<SimpleOrderedMap> result = (sp.requestedFields != null ? sp.requestedFields : new TreeSet<>(sp.schema.fields.keySet()))
+            .stream()
+            .map(sp.schema::getFieldOrNull)
+            .filter(it -> it != null)
+            .filter(it -> sp.includeDynamic || !sp.schema.isDynamicField(it.name))
+            .map(sp::getProperties)
+            .collect(Collectors.toList());
+        if (sp.includeDynamic && sp.requestedFields == null) {
+          result.addAll((Collection) Handler.DYNAMIC_FIELDS.fun.apply(sp));
+        }
+        return result;
+      }),
 
-  }
-  public SimpleOrderedMap<Object> getNamedPropertyValues(SolrParams params) {
-    SimpleOrderedMap<Object> topLevel = new SimpleOrderedMap<>();
-    topLevel.add(NAME, getSchemaName());
-    topLevel.add(VERSION, getVersion());
-    if (null != uniqueKeyFieldName) {
-      topLevel.add(UNIQUE_KEY, uniqueKeyFieldName);
-    }
-    if (null != defaultSearchFieldName) {
-      topLevel.add(DEFAULT_SEARCH_FIELD, defaultSearchFieldName);
-    }
-    if (isExplicitQueryParserDefaultOperator) {
-      SimpleOrderedMap<Object> solrQueryParserProperties = new SimpleOrderedMap<>();
-      solrQueryParserProperties.add(DEFAULT_OPERATOR, queryParserDefaultOperator);
-      topLevel.add(SOLR_QUERY_PARSER, solrQueryParserProperties);
-    }
-    if (isExplicitSimilarity) {
-      topLevel.add(SIMILARITY, similarityFactory.getNamedPropertyValues());
+
+      COPY_FIELDS(IndexSchema.COPY_FIELDS, sp -> sp.schema.getCopyFieldProperties(false,
+          sp.requestedSourceFields, sp.requestedDestinationFields));
+
+      final Function<SchemaProps, Object> fun;
+      final String name;
+      Handler(String name, Function<SchemaProps, Object> fun) {
+        this.fun = fun;
+        this.name = name;
+      }
     }
-    List<SimpleOrderedMap<Object>> fieldTypeProperties = new ArrayList<>();
-    SortedMap<String,FieldType> sortedFieldTypes = new TreeMap<>(fieldTypes);
-    for (FieldType fieldType : sortedFieldTypes.values()) {
-      fieldTypeProperties.add(fieldType.getNamedPropertyValues(params.getBool("showDefaults", false)));
+
+    SchemaProps(String name, SolrParams params, IndexSchema schema) {
+      this.name = name;
+      this.params = params;
+      this.schema = schema;
+      showDefaults = params.getBool("showDefaults", false);
+      includeDynamic = params.getBool("includeDynamic", false);
+
+      String sourceFieldListParam = params.get(SOURCE_FIELD_LIST);
+      if (null != sourceFieldListParam) {
+        String[] fields = sourceFieldListParam.trim().split("[,\\s]+");
+        if (fields.length > 0) {
+          requestedSourceFields = new HashSet<>(Arrays.asList(fields));
+          requestedSourceFields.remove(""); // Remove empty values, if any
+        }
+      }
+      String destinationFieldListParam = params.get(DESTINATION_FIELD_LIST);
+      if (null != destinationFieldListParam) {
+        String[] fields = destinationFieldListParam.trim().split("[,\\s]+");
+        if (fields.length > 0) {
+          requestedDestinationFields = new HashSet<>(Arrays.asList(fields));
+          requestedDestinationFields.remove(""); // Remove empty values, if any
+        }
+      }
+
+      String flParam = params.get(CommonParams.FL);
+      if (null != flParam) {
+        String[] fields = flParam.trim().split("[,\\s]+");
+        if (fields.length > 0)
+          requestedFields = new LinkedHashSet<>(Stream.of(fields)
+              .filter(it -> !it.trim().isEmpty())
+              .collect(Collectors.toList()));
+
+      }
+
     }
-    topLevel.add(FIELD_TYPES, fieldTypeProperties);  
-    List<SimpleOrderedMap<Object>> fieldProperties = new ArrayList<>();
-    SortedSet<String> fieldNames = new TreeSet<>(fields.keySet());
-    for (String fieldName : fieldNames) {
-      fieldProperties.add(fields.get(fieldName).getNamedPropertyValues(params.getBool("showDefaults", false)));
+
+
+    SimpleOrderedMap getProperties(SchemaField sf) {
+      SimpleOrderedMap<Object> result = sf.getNamedPropertyValues(showDefaults);
+      if (schema.isDynamicField(sf.name)) {
+        String dynamicBase = schema.getDynamicPattern(sf.getName());
+        // Add dynamicBase property if it's different from the field name.
+        if (!sf.getName().equals(dynamicBase)) {
+          result.add("dynamicBase", dynamicBase);
+        }
+      }
+      return result;
     }
-    if (params.getBool("includeDynamic", false)) {
-      fieldProperties.addAll(getDynamicFields(params));
+
+
+    @Override
+    public Map<String, Object> toMap() {
+      Map<String, Object> topLevel = new LinkedHashMap<>();
+      Stream.of(Handler.values())
+          .filter(it -> name == null || it.name.equals(name))
+          .forEach(it -> {
+            Object val = it.fun.apply(this);
+            if (val != null) topLevel.put(it.name, val);
+          });
+      return topLevel;
     }
-    topLevel.add(FIELDS, fieldProperties);
-    topLevel.add(DYNAMIC_FIELDS, getDynamicFields(params));
-    topLevel.add(COPY_FIELDS, getCopyFieldProperties(false, null, null));
-    return topLevel;
   }
 
-  private List<SimpleOrderedMap<Object>> getDynamicFields(SolrParams params) {
-    List<SimpleOrderedMap<Object>> dynamicFieldProperties = new ArrayList<>();
-    for (DynamicField dynamicField : dynamicFields) {
-      if ( ! dynamicField.getRegex().startsWith(INTERNAL_POLY_FIELD_PREFIX)) { // omit internal polyfields
-        dynamicFieldProperties.add(dynamicField.getPrototype().getNamedPropertyValues(params.getBool("showDefaults", false)));
-      }
-    }
-    return dynamicFieldProperties;
+  public Map<String, Object> getNamedPropertyValues(String name, SolrParams params) {
+    return new SchemaProps(name, params, this).toMap();
+
   }
 
+
   /**
    * Returns a list of copyField directives, with optional details and optionally restricting to those
    * directives that contain the requested source and/or destination field names.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e8cc19eb/solr/core/src/test/org/apache/solr/rest/schema/TestCopyFieldCollectionResource.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/rest/schema/TestCopyFieldCollectionResource.java b/solr/core/src/test/org/apache/solr/rest/schema/TestCopyFieldCollectionResource.java
index c0f936d..7b39ab3 100644
--- a/solr/core/src/test/org/apache/solr/rest/schema/TestCopyFieldCollectionResource.java
+++ b/solr/core/src/test/org/apache/solr/rest/schema/TestCopyFieldCollectionResource.java
@@ -100,4 +100,38 @@ public class TestCopyFieldCollectionResource extends SolrRestletTestBase {
 
   }
 
+  @Test
+  public void testRestrictSource() throws Exception {
+    assertQ("/schema/copyfields/?indent=on&wt=xml&source.fl=title,*_i,*_src_sub_i,src_sub_no_ast_i",
+        "count(/response/arr[@name='copyFields']/lst) = 16", // 4 + 4 + 4 + 4
+        "count(/response/arr[@name='copyFields']/lst/str[@name='source'][.='title']) = 4",
+        "count(/response/arr[@name='copyFields']/lst/str[@name='source'][.='*_i']) = 4",
+        "count(/response/arr[@name='copyFields']/lst/str[@name='source'][.='*_src_sub_i']) = 4",
+        "count(/response/arr[@name='copyFields']/lst/str[@name='source'][.='src_sub_no_ast_i']) = 4");
+  }
+
+  @Test
+  public void testRestrictDest() throws Exception {
+    assertQ("/schema/copyfields/?indent=on&wt=xml&dest.fl=title,*_s,*_dest_sub_s,dest_sub_no_ast_s",
+        "count(/response/arr[@name='copyFields']/lst) = 16", // 3 + 4 + 4 + 5
+        "count(/response/arr[@name='copyFields']/lst/str[@name='dest'][.='title']) = 3",
+        "count(/response/arr[@name='copyFields']/lst/str[@name='dest'][.='*_s']) = 4",
+        "count(/response/arr[@name='copyFields']/lst/str[@name='dest'][.='*_dest_sub_s']) = 4",
+        "count(/response/arr[@name='copyFields']/lst/str[@name='dest'][.='dest_sub_no_ast_s']) = 5");
+  }
+
+  @Test
+  public void testRestrictSourceAndDest() throws Exception {
+    assertQ("/schema/copyfields/?indent=on&wt=xml&source.fl=title,*_i&dest.fl=title,dest_sub_no_ast_s",
+        "count(/response/arr[@name='copyFields']/lst) = 3",
+
+        "/response/arr[@name='copyFields']/lst[    str[@name='source'][.='title']"
+            + "                                      and str[@name='dest'][.='dest_sub_no_ast_s']]",
+
+        "/response/arr[@name='copyFields']/lst[    str[@name='source'][.='*_i']"
+            + "                                      and str[@name='dest'][.='title']]",
+
+        "/response/arr[@name='copyFields']/lst[    str[@name='source'][.='*_i']"
+            + "                                      and str[@name='dest'][.='dest_sub_no_ast_s']]");
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e8cc19eb/solr/core/src/test/org/apache/solr/rest/schema/TestDynamicFieldCollectionResource.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/rest/schema/TestDynamicFieldCollectionResource.java b/solr/core/src/test/org/apache/solr/rest/schema/TestDynamicFieldCollectionResource.java
index 032bbad..6734ca2 100644
--- a/solr/core/src/test/org/apache/solr/rest/schema/TestDynamicFieldCollectionResource.java
+++ b/solr/core/src/test/org/apache/solr/rest/schema/TestDynamicFieldCollectionResource.java
@@ -30,10 +30,34 @@ public class TestDynamicFieldCollectionResource extends SolrRestletTestBase {
   }
 
   @Test
+  public void testGetTwoDynamicFields() throws IOException {
+    assertQ("/schema/dynamicfields?indent=on&wt=xml&fl=*_i,*_s",
+            "count(/response/arr[@name='dynamicFields']/lst/str[@name='name']) = 2",
+            "(/response/arr[@name='dynamicFields']/lst/str[@name='name'])[1] = '*_i'",
+            "(/response/arr[@name='dynamicFields']/lst/str[@name='name'])[2] = '*_s'");
+  }
+
+  @Test
+  public void testNotFoundDynamicFields() throws IOException {
+    assertQ("/schema/dynamicfields?indent=on&wt=xml&fl=*_not_in_there,this_one_isnt_either_*",
+            "count(/response/arr[@name='dynamicFields']) = 1",
+            "count(/response/arr[@name='dynamicfields']/lst/str[@name='name']) = 0");
+  }
+
+  @Test
   public void testJsonGetAllDynamicFields() throws Exception {
     assertJQ("/schema/dynamicfields?indent=on",
              "/dynamicFields/[0]/name=='*_coordinate'",
              "/dynamicFields/[1]/name=='ignored_*'",
              "/dynamicFields/[2]/name=='*_mfacet'");
   }
+
+  @Test
+  public void testJsonGetTwoDynamicFields() throws Exception {
+    assertJQ("/schema/dynamicfields?indent=on&fl=*_i,*_s&wt=xml", // assertJQ will fix the wt param to be json
+             "/dynamicFields/[0]/name=='*_i'",
+             "/dynamicFields/[1]/name=='*_s'");
+  }
+
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e8cc19eb/solr/core/src/test/org/apache/solr/rest/schema/TestFieldCollectionResource.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/rest/schema/TestFieldCollectionResource.java b/solr/core/src/test/org/apache/solr/rest/schema/TestFieldCollectionResource.java
index 6fdd298..f3de92b 100644
--- a/solr/core/src/test/org/apache/solr/rest/schema/TestFieldCollectionResource.java
+++ b/solr/core/src/test/org/apache/solr/rest/schema/TestFieldCollectionResource.java
@@ -15,6 +15,8 @@
  * limitations under the License.
  */
 package org.apache.solr.rest.schema;
+import java.io.IOException;
+
 import org.apache.solr.rest.SolrRestletTestBase;
 import org.junit.Test;
 
@@ -37,6 +39,37 @@ public class TestFieldCollectionResource extends SolrRestletTestBase {
              "/fields/[2]/name=='_version_'");
   }
 
+  @Test
+  public void testGetThreeFieldsDontIncludeDynamic() throws IOException {
+    //
+    assertQ("/schema/fields?indent=on&wt=xml&fl=id,_version_,price_i",
+        "count(/response/arr[@name='fields']/lst/str[@name='name']) = 2",
+        "(/response/arr[@name='fields']/lst/str[@name='name'])[1] = 'id'",
+        "(/response/arr[@name='fields']/lst/str[@name='name'])[2] = '_version_'");
+  }
+
+  @Test
+  public void testGetThreeFieldsIncludeDynamic() throws IOException {
+    assertQ("/schema/fields?indent=on&wt=xml&fl=id,_version_,price_i&includeDynamic=on",
+
+        "count(/response/arr[@name='fields']/lst/str[@name='name']) = 3",
+
+        "(/response/arr[@name='fields']/lst/str[@name='name'])[1] = 'id'",
+
+        "(/response/arr[@name='fields']/lst/str[@name='name'])[2] = '_version_'",
+
+        "(/response/arr[@name='fields']/lst/str[@name='name'])[3] = 'price_i'",
+
+        "/response/arr[@name='fields']/lst[    str[@name='name']='price_i'    "
+            +"                                  and str[@name='dynamicBase']='*_i']");
+  }
+  @Test
+  public void testNotFoundFields() throws IOException {
+    assertQ("/schema/fields?indent=on&wt=xml&fl=not_in_there,this_one_either",
+        "count(/response/arr[@name='fields']) = 1",
+        "count(/response/arr[@name='fields']/lst/str[@name='name']) = 0");
+  }
+
 
   @Test
   public void testJsonGetAllFieldsIncludeDynamic() throws Exception {


[30/50] [abbrv] lucene-solr:solr-5750: SOLR-9014: Deprecate and reduce usage of ClusterState methods which may make calls to ZK via the lazy collection reference

Posted by ds...@apache.org.
SOLR-9014: Deprecate and reduce usage of ClusterState methods which may make calls to ZK via the lazy collection reference


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

Branch: refs/heads/solr-5750
Commit: 922265b478296992189434040517368cf93d1b09
Parents: 837264a
Author: Shalin Shekhar Mangar <sh...@apache.org>
Authored: Tue Apr 26 01:50:26 2016 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Tue Apr 26 01:50:26 2016 +0530

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  3 ++
 .../src/java/org/apache/solr/cloud/Assign.java  |  8 ++--
 .../OverseerAutoReplicaFailoverThread.java      |  1 +
 .../cloud/OverseerCollectionMessageHandler.java | 40 ++++++++--------
 .../cloud/OverseerConfigSetMessageHandler.java  |  1 +
 .../cloud/overseer/ClusterStateMutator.java     | 16 +++----
 .../solr/cloud/overseer/CollectionMutator.java  |  6 +--
 .../apache/solr/cloud/overseer/NodeMutator.java |  9 ++--
 .../solr/cloud/overseer/ReplicaMutator.java     | 36 ++++++++-------
 .../solr/cloud/overseer/SliceMutator.java       | 39 ++++++++--------
 .../solr/cloud/overseer/ZkStateWriter.java      |  3 +-
 .../src/java/org/apache/solr/core/SolrCore.java |  5 +-
 .../solr/handler/CdcrUpdateLogSynchronizer.java |  4 +-
 .../org/apache/solr/handler/SQLHandler.java     |  1 +
 .../solr/handler/admin/CollectionsHandler.java  | 27 ++++++-----
 .../solr/handler/admin/CoreAdminOperation.java  | 11 +++--
 .../org/apache/solr/servlet/HttpSolrCall.java   | 11 +++--
 .../processor/DistributedUpdateProcessor.java   |  9 ++--
 .../src/java/org/apache/solr/util/SolrCLI.java  |  1 +
 .../org/apache/solr/util/SolrLogLayout.java     | 10 ++--
 .../test/org/apache/solr/cloud/AssignTest.java  |  2 +-
 .../cloud/CollectionsAPIDistributedZkTest.java  |  8 ++--
 .../solr/cloud/CollectionsAPISolrJTest.java     |  2 +-
 .../org/apache/solr/cloud/OverseerTest.java     |  3 +-
 .../org/apache/solr/handler/TestSQLHandler.java |  1 +
 .../client/solrj/io/stream/TopicStream.java     |  1 +
 .../apache/solr/common/cloud/ClusterState.java  | 48 ++++++++++++++------
 .../solr/common/cloud/ClusterStateUtil.java     |  1 +
 .../apache/solr/common/cloud/DocCollection.java |  6 +++
 .../apache/solr/common/cloud/ZkStateReader.java |  3 +-
 .../solr/client/solrj/io/sql/JdbcTest.java      |  1 +
 31 files changed, 186 insertions(+), 131 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 6034851..999bd73 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -169,6 +169,9 @@ Optimizations
 
 * SOLR-8973: Zookeeper frenzy when a core is first created. (Janmejay Singh, Scott Blum, shalin)
 
+* SOLR-9014: Deprecate and reduce usage of ClusterState methods which may make calls to ZK via
+  the lazy collection reference. (Scott Blum, shalin)
+
 Other Changes
 ----------------------
 * SOLR-7516: Improve javadocs for JavaBinCodec, ObjectResolver and enforce the single-usage policy.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/cloud/Assign.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/Assign.java b/solr/core/src/java/org/apache/solr/cloud/Assign.java
index 6939270..92310ed 100644
--- a/solr/core/src/java/org/apache/solr/cloud/Assign.java
+++ b/solr/core/src/java/org/apache/solr/cloud/Assign.java
@@ -46,8 +46,8 @@ import static org.apache.solr.common.cloud.ZkStateReader.MAX_SHARDS_PER_NODE;
 public class Assign {
   private static Pattern COUNT = Pattern.compile("core_node(\\d+)");
 
-  public static String assignNode(String collection, ClusterState state) {
-    Map<String, Slice> sliceMap = state.getSlicesMap(collection);
+  public static String assignNode(DocCollection collection) {
+    Map<String, Slice> sliceMap = collection != null ? collection.getSlicesMap() : null;
     if (sliceMap == null) {
       return "core_node1";
     }
@@ -70,12 +70,12 @@ public class Assign {
    *
    * @return the assigned shard id
    */
-  public static String assignShard(String collection, ClusterState state, Integer numShards) {
+  public static String assignShard(DocCollection collection, Integer numShards) {
     if (numShards == null) {
       numShards = 1;
     }
     String returnShardId = null;
-    Map<String, Slice> sliceMap = state.getActiveSlicesMap(collection);
+    Map<String, Slice> sliceMap = collection != null ? collection.getActiveSlicesMap() : null;
 
 
     // TODO: now that we create shards ahead of time, is this code needed?  Esp since hash ranges aren't assigned when creating via this method?

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java b/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java
index f94ffcc..e8ac6c5 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java
@@ -24,6 +24,7 @@ import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeMap;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java b/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
index 6bff648..ce04841 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
@@ -945,16 +945,13 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
                 + router.getClass().getName());
       }
     } else {
-      parentSlice = clusterState.getSlice(collectionName, slice);
+      parentSlice = collection.getSlice(slice);
     }
     
     if (parentSlice == null) {
-      if (clusterState.hasCollection(collectionName)) {
-        throw new SolrException(ErrorCode.BAD_REQUEST, "No shard with the specified name exists: " + slice);
-      } else {
-        throw new SolrException(ErrorCode.BAD_REQUEST,
-            "No collection with the specified name exists: " + collectionName);
-      }
+      // no chance of the collection being null because ClusterState#getCollection(String) would have thrown
+      // an exception already
+      throw new SolrException(ErrorCode.BAD_REQUEST, "No shard with the specified name exists: " + slice);
     }
     
     // find the leader for the shard
@@ -1039,7 +1036,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
         String subShardName = collectionName + "_" + subSlice + "_replica1";
         subShardNames.add(subShardName);
         
-        Slice oSlice = clusterState.getSlice(collectionName, subSlice);
+        Slice oSlice = collection.getSlice(subSlice);
         if (oSlice != null) {
           final Slice.State state = oSlice.getState();
           if (state == Slice.State.ACTIVE) {
@@ -1180,7 +1177,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
       
       // TODO: Have replication factor decided in some other way instead of numShards for the parent
       
-      int repFactor = clusterState.getSlice(collectionName, slice).getReplicas().size();
+      int repFactor = parentSlice.getReplicas().size();
       
       // we need to look at every node and see how many cores it serves
       // add our new cores to existing nodes serving the least number of cores
@@ -1379,18 +1376,18 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
   }
 
   private void deleteShard(ClusterState clusterState, ZkNodeProps message, NamedList results) {
-    String collection = message.getStr(ZkStateReader.COLLECTION_PROP);
+    String collectionName = message.getStr(ZkStateReader.COLLECTION_PROP);
     String sliceId = message.getStr(ZkStateReader.SHARD_ID_PROP);
     
     log.info("Delete shard invoked");
-    Slice slice = clusterState.getSlice(collection, sliceId);
+    Slice slice = clusterState.getSlice(collectionName, sliceId);
     
     if (slice == null) {
-      if (clusterState.hasCollection(collection)) {
+      if (clusterState.hasCollection(collectionName)) {
         throw new SolrException(ErrorCode.BAD_REQUEST,
-            "No shard with name " + sliceId + " exists for collection " + collection);
+            "No shard with name " + sliceId + " exists for collection " + collectionName);
       } else {
-        throw new SolrException(ErrorCode.BAD_REQUEST, "No collection with the specified name exists: " + collection);
+        throw new SolrException(ErrorCode.BAD_REQUEST, "No collection with the specified name exists: " + collectionName);
       }
     }
     // For now, only allow for deletions of Inactive slices or custom hashes (range==null).
@@ -1421,7 +1418,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
       processResponses(results, shardHandler, true, "Failed to delete shard", asyncId, requestMap, Collections.emptySet());
 
       ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, DELETESHARD.toLower(), ZkStateReader.COLLECTION_PROP,
-          collection, ZkStateReader.SHARD_ID_PROP, sliceId);
+          collectionName, ZkStateReader.SHARD_ID_PROP, sliceId);
       Overseer.getStateUpdateQueue(zkStateReader.getZkClient()).offer(Utils.toJSON(m));
       
       // wait for a while until we don't see the shard
@@ -1429,7 +1426,8 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
       boolean removed = false;
       while (! timeout.hasTimedOut()) {
         Thread.sleep(100);
-        removed = zkStateReader.getClusterState().getSlice(collection, sliceId) == null;
+        DocCollection collection = zkStateReader.getClusterState().getCollection(collectionName);
+        removed = collection.getSlice(sliceId) == null;
         if (removed) {
           Thread.sleep(100); // just a bit of time so it's more likely other readers see on return
           break;
@@ -1437,16 +1435,16 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
       }
       if (!removed) {
         throw new SolrException(ErrorCode.SERVER_ERROR,
-            "Could not fully remove collection: " + collection + " shard: " + sliceId);
+            "Could not fully remove collection: " + collectionName + " shard: " + sliceId);
       }
       
-      log.info("Successfully deleted collection: " + collection + ", shard: " + sliceId);
+      log.info("Successfully deleted collection: " + collectionName + ", shard: " + sliceId);
       
     } catch (SolrException e) {
       throw e;
     } catch (Exception e) {
       throw new SolrException(ErrorCode.SERVER_ERROR,
-          "Error executing delete operation for collection: " + collection + " shard: " + sliceId, e);
+          "Error executing delete operation for collection: " + collectionName + " shard: " + sliceId, e);
     }
   }
 
@@ -1561,7 +1559,9 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
     boolean added = false;
     while (! waitUntil.hasTimedOut()) {
       Thread.sleep(100);
-      Map<String, RoutingRule> rules = zkStateReader.getClusterState().getSlice(sourceCollection.getName(), sourceSlice.getName()).getRoutingRules();
+      sourceCollection = zkStateReader.getClusterState().getCollection(sourceCollection.getName());
+      sourceSlice = sourceCollection.getSlice(sourceSlice.getName());
+      Map<String, RoutingRule> rules = sourceSlice.getRoutingRules();
       if (rules != null) {
         RoutingRule rule = rules.get(SolrIndexSplitter.getRouteKey(splitKey) + "!");
         if (rule != null && rule.getRouteRanges().contains(splitRange)) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/cloud/OverseerConfigSetMessageHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerConfigSetMessageHandler.java b/solr/core/src/java/org/apache/solr/cloud/OverseerConfigSetMessageHandler.java
index 1f972ff..15fed42 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerConfigSetMessageHandler.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerConfigSetMessageHandler.java
@@ -30,6 +30,7 @@ import java.util.Set;
 import org.apache.solr.client.solrj.SolrResponse;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkConfigManager;
 import org.apache.solr.common.cloud.ZkNodeProps;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/cloud/overseer/ClusterStateMutator.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/ClusterStateMutator.java b/solr/core/src/java/org/apache/solr/cloud/overseer/ClusterStateMutator.java
index 7ffa8c1..0a76d91 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/ClusterStateMutator.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/ClusterStateMutator.java
@@ -122,7 +122,7 @@ public class ClusterStateMutator {
   public static ClusterState newState(ClusterState state, String name, DocCollection collection) {
     ClusterState newClusterState = null;
     if (collection == null) {
-      newClusterState = state.copyWith(name, (DocCollection) null);
+      newClusterState = state.copyWith(name, null);
     } else {
       newClusterState = state.copyWith(name, collection);
     }
@@ -153,9 +153,8 @@ public class ClusterStateMutator {
   /*
        * Return an already assigned id or null if not assigned
        */
-  public static String getAssignedId(final ClusterState state, final String nodeName,
-                              final ZkNodeProps coreState) {
-    Collection<Slice> slices = state.getSlices(coreState.getStr(ZkStateReader.COLLECTION_PROP));
+  public static String getAssignedId(final DocCollection collection, final String nodeName) {
+    Collection<Slice> slices = collection != null ? collection.getSlices() : null;
     if (slices != null) {
       for (Slice slice : slices) {
         if (slice.getReplicasMap().get(nodeName) != null) {
@@ -166,18 +165,15 @@ public class ClusterStateMutator {
     return null;
   }
 
-  public static String getAssignedCoreNodeName(ClusterState state, ZkNodeProps message) {
-    Collection<Slice> slices = state.getSlices(message.getStr(ZkStateReader.COLLECTION_PROP));
+  public static String getAssignedCoreNodeName(DocCollection collection, String forNodeName, String forCoreName) {
+    Collection<Slice> slices = collection != null ? collection.getSlices() : null;
     if (slices != null) {
       for (Slice slice : slices) {
         for (Replica replica : slice.getReplicas()) {
           String nodeName = replica.getStr(ZkStateReader.NODE_NAME_PROP);
           String core = replica.getStr(ZkStateReader.CORE_NAME_PROP);
 
-          String msgNodeName = message.getStr(ZkStateReader.NODE_NAME_PROP);
-          String msgCore = message.getStr(ZkStateReader.CORE_NAME_PROP);
-
-          if (nodeName.equals(msgNodeName) && core.equals(msgCore)) {
+          if (nodeName.equals(forNodeName) && core.equals(forCoreName)) {
             return replica.getName();
           }
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/cloud/overseer/CollectionMutator.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/CollectionMutator.java b/solr/core/src/java/org/apache/solr/cloud/overseer/CollectionMutator.java
index 4f7cb52..3d950fe 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/CollectionMutator.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/CollectionMutator.java
@@ -51,7 +51,8 @@ public class CollectionMutator {
     String collectionName = message.getStr(ZkStateReader.COLLECTION_PROP);
     if (!checkCollectionKeyExistence(message)) return ZkStateWriter.NO_OP;
     String shardId = message.getStr(ZkStateReader.SHARD_ID_PROP);
-    Slice slice = clusterState.getSlice(collectionName, shardId);
+    DocCollection collection = clusterState.getCollection(collectionName);
+    Slice slice = collection.getSlice(shardId);
     if (slice == null) {
       Map<String, Replica> replicas = Collections.EMPTY_MAP;
       Map<String, Object> sliceProps = new HashMap<>();
@@ -63,8 +64,7 @@ public class CollectionMutator {
       if (shardParent != null) {
         sliceProps.put(Slice.PARENT, shardParent);
       }
-      DocCollection collection = updateSlice(collectionName,
-          clusterState.getCollection(collectionName), new Slice(shardId, replicas, sliceProps));
+      collection = updateSlice(collectionName, collection, new Slice(shardId, replicas, sliceProps));
       return new ZkWriteCommand(collectionName, collection);
     } else {
       log.error("Unable to create Shard: " + shardId + " because it already exists in collection: " + collectionName);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/cloud/overseer/NodeMutator.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/NodeMutator.java b/solr/core/src/java/org/apache/solr/cloud/overseer/NodeMutator.java
index 0784cd4..5dd27c1 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/NodeMutator.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/NodeMutator.java
@@ -27,6 +27,7 @@ import java.util.Set;
 import java.util.Map.Entry;
 
 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.ZkNodeProps;
@@ -50,12 +51,12 @@ public class NodeMutator {
 
     Set<String> collections = clusterState.getCollections();
     for (String collection : collections) {
-
-      Map<String,Slice> slicesCopy = new LinkedHashMap<>(clusterState.getSlicesMap(collection));
+      DocCollection docCollection = clusterState.getCollection(collection);
+      Map<String,Slice> slicesCopy = new LinkedHashMap<>(docCollection.getSlicesMap());
 
       Set<Entry<String,Slice>> entries = slicesCopy.entrySet();
       for (Entry<String,Slice> entry : entries) {
-        Slice slice = clusterState.getSlice(collection, entry.getKey());
+        Slice slice = docCollection.getSlice(entry.getKey());
         Map<String,Replica> newReplicas = new HashMap<String,Replica>();
 
         Collection<Replica> replicas = slice.getReplicas();
@@ -76,7 +77,7 @@ public class NodeMutator {
 
       }
 
-      zkWriteCommands.add(new ZkWriteCommand(collection, clusterState.getCollection(collection).copyWithSlices(slicesCopy)));
+      zkWriteCommands.add(new ZkWriteCommand(collection, docCollection.copyWithSlices(slicesCopy)));
     }
 
     return zkWriteCommands;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java b/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java
index a3efc80..5147f43 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java
@@ -127,7 +127,8 @@ public class ReplicaMutator {
       isUnique = Boolean.parseBoolean(shardUnique);
     }
 
-    Replica replica = clusterState.getReplica(collectionName, replicaName);
+    DocCollection collection = clusterState.getCollection(collectionName);
+    Replica replica = collection.getReplica(replicaName);
 
     if (replica == null) {
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Could not find collection/slice/replica " +
@@ -138,7 +139,7 @@ public class ReplicaMutator {
     if (StringUtils.equalsIgnoreCase(replica.getStr(property), propVal)) return ZkStateWriter.NO_OP; // already the value we're going to set
 
     // OK, there's no way we won't change the cluster state now
-    Map<String,Replica> replicas = clusterState.getSlice(collectionName, sliceName).getReplicasCopy();
+    Map<String,Replica> replicas = collection.getSlice(sliceName).getReplicasCopy();
     if (isUnique == false) {
       replicas.get(replicaName).getProperties().put(property, propVal);
     } else { // Set prop for this replica, but remove it for all others.
@@ -150,8 +151,8 @@ public class ReplicaMutator {
         }
       }
     }
-    Slice newSlice = new Slice(sliceName, replicas, clusterState.getSlice(collectionName, sliceName).shallowCopy());
-    DocCollection newCollection = CollectionMutator.updateSlice(collectionName, clusterState.getCollection(collectionName),
+    Slice newSlice = new Slice(sliceName, replicas, collection.getSlice(sliceName).shallowCopy());
+    DocCollection newCollection = CollectionMutator.updateSlice(collectionName, collection,
         newSlice);
     return new ZkWriteCommand(collectionName, newCollection);
   }
@@ -174,7 +175,8 @@ public class ReplicaMutator {
       property = OverseerCollectionMessageHandler.COLL_PROP_PREFIX + property;
     }
 
-    Replica replica = clusterState.getReplica(collectionName, replicaName);
+    DocCollection collection = clusterState.getCollection(collectionName);
+    Replica replica = collection.getReplica(replicaName);
 
     if (replica == null) {
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Could not find collection/slice/replica " +
@@ -188,7 +190,6 @@ public class ReplicaMutator {
 
     log.info("Deleting property " + property + " for collection: " + collectionName +
         " slice " + sliceName + " replica " + replicaName + ". Full message: " + message);
-    DocCollection collection = clusterState.getCollection(collectionName);
     Slice slice = collection.getSlice(sliceName);
     DocCollection newCollection = SliceMutator.updateReplica(collection,
         slice, replicaName, unsetProperty(replica, property));
@@ -232,13 +233,15 @@ public class ReplicaMutator {
     String sliceName = message.getStr(ZkStateReader.SHARD_ID_PROP);
 
     String coreNodeName = message.getStr(ZkStateReader.CORE_NODE_NAME_PROP);
+    DocCollection collection = prevState.getCollectionOrNull(collectionName);
     if (coreNodeName == null) {
-      coreNodeName = ClusterStateMutator.getAssignedCoreNodeName(prevState, message);
+      coreNodeName = ClusterStateMutator.getAssignedCoreNodeName(collection,
+          message.getStr(ZkStateReader.NODE_NAME_PROP), message.getStr(ZkStateReader.CORE_NAME_PROP));
       if (coreNodeName != null) {
         log.info("node=" + coreNodeName + " is already registered");
       } else {
         // if coreNodeName is null, auto assign one
-        coreNodeName = Assign.assignNode(collectionName, prevState);
+        coreNodeName = Assign.assignNode(collection);
       }
       message.getProperties().put(ZkStateReader.CORE_NODE_NAME_PROP,
           coreNodeName);
@@ -247,7 +250,7 @@ public class ReplicaMutator {
     // use the provided non null shardId
     if (sliceName == null) {
       //get shardId from ClusterState
-      sliceName = ClusterStateMutator.getAssignedId(prevState, coreNodeName, message);
+      sliceName = ClusterStateMutator.getAssignedId(collection, coreNodeName);
       if (sliceName != null) {
         log.info("shard=" + sliceName + " is already registered");
       }
@@ -256,14 +259,14 @@ public class ReplicaMutator {
       //request new shardId
       if (collectionExists) {
         // use existing numShards
-        numShards = prevState.getCollection(collectionName).getSlices().size();
+        numShards = collection.getSlices().size();
         log.info("Collection already exists with " + ZkStateReader.NUM_SHARDS_PROP + "=" + numShards);
       }
-      sliceName = Assign.assignShard(collectionName, prevState, numShards);
+      sliceName = Assign.assignShard(collection, numShards);
       log.info("Assigning new node to shard shard=" + sliceName);
     }
 
-    Slice slice = prevState.getSlice(collectionName, sliceName);
+    Slice slice = collection != null ?  collection.getSlice(sliceName) : null;
 
     Map<String, Object> replicaProps = new LinkedHashMap<>();
 
@@ -313,9 +316,7 @@ public class ReplicaMutator {
     Map<String, Object> sliceProps = null;
     Map<String, Replica> replicas;
 
-    DocCollection collection = prevState.getCollectionOrNull(collectionName);
     if (slice != null) {
-      collection = prevState.getCollection(collectionName);
       collection = checkAndCompleteShardSplit(prevState, collection, coreNodeName, sliceName, replica);
       // get the current slice again because it may have been updated due to checkAndCompleteShardSplit method
       slice = collection.getSlice(sliceName);
@@ -340,15 +341,16 @@ public class ReplicaMutator {
    * Handles non-legacy state updates
    */
   protected ZkWriteCommand updateStateNew(ClusterState clusterState, final ZkNodeProps message) {
-    String collection = message.getStr(ZkStateReader.COLLECTION_PROP);
+    String collectionName = message.getStr(ZkStateReader.COLLECTION_PROP);
     if (!checkCollectionKeyExistence(message)) return ZkStateWriter.NO_OP;
     String sliceName = message.getStr(ZkStateReader.SHARD_ID_PROP);
 
-    if (collection == null || sliceName == null) {
+    if (collectionName == null || sliceName == null) {
       log.error("Invalid collection and slice {}", message);
       return ZkStateWriter.NO_OP;
     }
-    Slice slice = clusterState.getSlice(collection, sliceName);
+    DocCollection collection = clusterState.getCollectionOrNull(collectionName);
+    Slice slice = collection != null ? collection.getSlice(sliceName) : null;
     if (slice == null) {
       log.error("No such slice exists {}", message);
       return ZkStateWriter.NO_OP;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/cloud/overseer/SliceMutator.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/SliceMutator.java b/solr/core/src/java/org/apache/solr/cloud/overseer/SliceMutator.java
index 63c37ef..836d014 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/SliceMutator.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/SliceMutator.java
@@ -69,7 +69,7 @@ public class SliceMutator {
       return ZkStateWriter.NO_OP;
     }
 
-    String coreNodeName = Assign.assignNode(coll, clusterState);
+    String coreNodeName = Assign.assignNode(collection);
     Replica replica = new Replica(coreNodeName,
         makeMap(
             ZkStateReader.CORE_NAME_PROP, message.getStr(ZkStateReader.CORE_NAME_PROP),
@@ -149,18 +149,19 @@ public class SliceMutator {
   }
 
   public ZkWriteCommand updateShardState(ClusterState clusterState, ZkNodeProps message) {
-    String collection = message.getStr(ZkStateReader.COLLECTION_PROP);
+    String collectionName = message.getStr(ZkStateReader.COLLECTION_PROP);
     if (!checkCollectionKeyExistence(message)) return ZkStateWriter.NO_OP;
-    log.info("Update shard state invoked for collection: " + collection + " with message: " + message);
+    log.info("Update shard state invoked for collection: " + collectionName + " with message: " + message);
 
-    Map<String, Slice> slicesCopy = new LinkedHashMap<>(clusterState.getSlicesMap(collection));
+    DocCollection collection = clusterState.getCollection(collectionName);
+    Map<String, Slice> slicesCopy = new LinkedHashMap<>(collection.getSlicesMap());
     for (String key : message.keySet()) {
       if (ZkStateReader.COLLECTION_PROP.equals(key)) continue;
       if (Overseer.QUEUE_OPERATION.equals(key)) continue;
 
-      Slice slice = clusterState.getSlice(collection, key);
+      Slice slice = collection.getSlice(key);
       if (slice == null) {
-        throw new RuntimeException("Overseer.updateShardState unknown collection: " + collection + " slice: " + key);
+        throw new RuntimeException("Overseer.updateShardState unknown collection: " + collectionName + " slice: " + key);
       }
       log.info("Update shard state " + key + " to " + message.getStr(key));
       Map<String, Object> props = slice.shallowCopy();
@@ -174,11 +175,11 @@ public class SliceMutator {
       slicesCopy.put(slice.getName(), newSlice);
     }
 
-    return new ZkWriteCommand(collection, clusterState.getCollection(collection).copyWithSlices(slicesCopy));
+    return new ZkWriteCommand(collectionName, collection.copyWithSlices(slicesCopy));
   }
 
   public ZkWriteCommand addRoutingRule(final ClusterState clusterState, ZkNodeProps message) {
-    String collection = message.getStr(ZkStateReader.COLLECTION_PROP);
+    String collectionName = message.getStr(ZkStateReader.COLLECTION_PROP);
     if (!checkCollectionKeyExistence(message)) return ZkStateWriter.NO_OP;
     String shard = message.getStr(ZkStateReader.SHARD_ID_PROP);
     String routeKey = message.getStr("routeKey");
@@ -187,9 +188,10 @@ public class SliceMutator {
     String targetShard = message.getStr("targetShard");
     String expireAt = message.getStr("expireAt");
 
-    Slice slice = clusterState.getSlice(collection, shard);
+    DocCollection collection = clusterState.getCollection(collectionName);
+    Slice slice = collection.getSlice(shard);
     if (slice == null) {
-      throw new RuntimeException("Overseer.addRoutingRule unknown collection: " + collection + " slice:" + shard);
+      throw new RuntimeException("Overseer.addRoutingRule unknown collection: " + collectionName + " slice:" + shard);
     }
 
     Map<String, RoutingRule> routingRules = slice.getRoutingRules();
@@ -215,22 +217,23 @@ public class SliceMutator {
     props.put("routingRules", routingRules);
 
     Slice newSlice = new Slice(slice.getName(), slice.getReplicasCopy(), props);
-    return new ZkWriteCommand(collection,
-        CollectionMutator.updateSlice(collection, clusterState.getCollection(collection), newSlice));
+    return new ZkWriteCommand(collectionName,
+        CollectionMutator.updateSlice(collectionName, collection, newSlice));
   }
 
   public ZkWriteCommand removeRoutingRule(final ClusterState clusterState, ZkNodeProps message) {
-    String collection = message.getStr(ZkStateReader.COLLECTION_PROP);
+    String collectionName = message.getStr(ZkStateReader.COLLECTION_PROP);
     if (!checkCollectionKeyExistence(message)) return ZkStateWriter.NO_OP;
     String shard = message.getStr(ZkStateReader.SHARD_ID_PROP);
     String routeKeyStr = message.getStr("routeKey");
 
-    log.info("Overseer.removeRoutingRule invoked for collection: " + collection
+    log.info("Overseer.removeRoutingRule invoked for collection: " + collectionName
         + " shard: " + shard + " routeKey: " + routeKeyStr);
 
-    Slice slice = clusterState.getSlice(collection, shard);
+    DocCollection collection = clusterState.getCollection(collectionName);
+    Slice slice = collection.getSlice(shard);
     if (slice == null) {
-      log.warn("Unknown collection: " + collection + " shard: " + shard);
+      log.warn("Unknown collection: " + collectionName + " shard: " + shard);
       return ZkStateWriter.NO_OP;
     }
     Map<String, RoutingRule> routingRules = slice.getRoutingRules();
@@ -239,8 +242,8 @@ public class SliceMutator {
       Map<String, Object> props = slice.shallowCopy();
       props.put("routingRules", routingRules);
       Slice newSlice = new Slice(slice.getName(), slice.getReplicasCopy(), props);
-      return new ZkWriteCommand(collection,
-          CollectionMutator.updateSlice(collection, clusterState.getCollection(collection), newSlice));
+      return new ZkWriteCommand(collectionName,
+          CollectionMutator.updateSlice(collectionName, collection, newSlice));
     }
 
     return ZkStateWriter.NO_OP;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/cloud/overseer/ZkStateWriter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/ZkStateWriter.java b/solr/core/src/java/org/apache/solr/cloud/overseer/ZkStateWriter.java
index ec67ed7..e9edef1 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/ZkStateWriter.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/ZkStateWriter.java
@@ -18,6 +18,7 @@ package org.apache.solr.cloud.overseer;
 
 import java.lang.invoke.MethodHandles;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
@@ -227,8 +228,8 @@ public class ZkStateWriter {
           } else if (c.getStateFormat() > 1) {
             byte[] data = Utils.toJSON(singletonMap(c.getName(), c));
             if (reader.getZkClient().exists(path, true)) {
-              assert c.getZNodeVersion() >= 0;
               log.info("going to update_collection {} version: {}", path, c.getZNodeVersion());
+              assert c.getZNodeVersion() >= 0;
               Stat stat = reader.getZkClient().setData(path, data, c.getZNodeVersion(), true);
               DocCollection newCollection = new DocCollection(name, c.getSlicesMap(), c.getProperties(), c.getRouter(), stat.getVersion(), path);
               clusterState = clusterState.copyWith(name, newCollection);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/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 bb0cd05..b94b3d8 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -55,6 +55,7 @@ import org.apache.solr.cloud.CloudDescriptor;
 import org.apache.solr.cloud.ZkSolrResourceLoader;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.params.CommonParams;
@@ -834,8 +835,8 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
 
       // ZK pre-register would have already happened so we read slice properties now
       final ClusterState clusterState = cc.getZkController().getClusterState();
-      final Slice slice = clusterState.getSlice(coreDescriptor.getCloudDescriptor().getCollectionName(), 
-          coreDescriptor.getCloudDescriptor().getShardId());
+      final DocCollection collection = clusterState.getCollection(coreDescriptor.getCloudDescriptor().getCollectionName());
+      final Slice slice = collection.getSlice(coreDescriptor.getCloudDescriptor().getShardId());
       if (slice.getState() == Slice.State.CONSTRUCTION) {
         // set update log to buffer before publishing the core
         getUpdateHandler().getUpdateLog().bufferUpdates();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/handler/CdcrUpdateLogSynchronizer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/CdcrUpdateLogSynchronizer.java b/solr/core/src/java/org/apache/solr/handler/CdcrUpdateLogSynchronizer.java
index 7071908..48bfec0 100644
--- a/solr/core/src/java/org/apache/solr/handler/CdcrUpdateLogSynchronizer.java
+++ b/solr/core/src/java/org/apache/solr/handler/CdcrUpdateLogSynchronizer.java
@@ -28,6 +28,7 @@ import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.ZkCoreNodeProps;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.params.CommonParams;
@@ -113,7 +114,8 @@ class CdcrUpdateLogSynchronizer implements CdcrStateManager.CdcrStateObserver {
     private String getLeaderUrl() {
       ZkController zkController = core.getCoreDescriptor().getCoreContainer().getZkController();
       ClusterState cstate = zkController.getClusterState();
-      ZkNodeProps leaderProps = cstate.getLeader(collection, shardId);
+      DocCollection docCollection = cstate.getCollection(collection);
+      ZkNodeProps leaderProps = docCollection.getLeader(shardId);
       if (leaderProps == null) { // we might not have a leader yet, returns null
         return null;
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/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 aa40046..c26c5a8 100644
--- a/solr/core/src/java/org/apache/solr/handler/SQLHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/SQLHandler.java
@@ -55,6 +55,7 @@ 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;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
index 64b10ab..edafc54 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
@@ -425,7 +425,8 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
 
         ClusterState clusterState = h.coreContainer.getZkController().getClusterState();
 
-        ZkNodeProps leaderProps = clusterState.getLeader(collection, shard);
+        DocCollection docCollection = clusterState.getCollection(collection);
+        ZkNodeProps leaderProps = docCollection.getLeader(shard);
         ZkCoreNodeProps nodeProps = new ZkCoreNodeProps(leaderProps);
 
         try (HttpSolrClient client = new Builder(nodeProps.getBaseUrl()).build()) {
@@ -828,18 +829,15 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
 
   private static void forceLeaderElection(SolrQueryRequest req, CollectionsHandler handler) {
     ClusterState clusterState = handler.coreContainer.getZkController().getClusterState();
-    String collection = req.getParams().required().get(COLLECTION_PROP);
+    String collectionName = req.getParams().required().get(COLLECTION_PROP);
     String sliceId = req.getParams().required().get(SHARD_ID_PROP);
 
     log.info("Force leader invoked, state: {}", clusterState);
-    Slice slice = clusterState.getSlice(collection, sliceId);
+    DocCollection collection = clusterState.getCollection(collectionName);
+    Slice slice = collection.getSlice(sliceId);
     if (slice == null) {
-      if (clusterState.hasCollection(collection)) {
-        throw new SolrException(ErrorCode.BAD_REQUEST,
-            "No shard with name " + sliceId + " exists for collection " + collection);
-      } else {
-        throw new SolrException(ErrorCode.BAD_REQUEST, "No collection with the specified name exists: " + collection);
-      }
+      throw new SolrException(ErrorCode.BAD_REQUEST,
+          "No shard with name " + sliceId + " exists for collection " + collectionName);
     }
 
     try {
@@ -851,7 +849,7 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
       }
 
       // Clear out any LIR state
-      String lirPath = handler.coreContainer.getZkController().getLeaderInitiatedRecoveryZnodePath(collection, sliceId);
+      String lirPath = handler.coreContainer.getZkController().getLeaderInitiatedRecoveryZnodePath(collectionName, sliceId);
       if (handler.coreContainer.getZkController().getZkClient().exists(lirPath, true)) {
         StringBuilder sb = new StringBuilder();
         handler.coreContainer.getZkController().getZkClient().printLayout(lirPath, 4, sb);
@@ -880,7 +878,8 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
       for (int i = 0; i < 9; i++) {
         Thread.sleep(5000);
         clusterState = handler.coreContainer.getZkController().getClusterState();
-        slice = clusterState.getSlice(collection, sliceId);
+        collection = clusterState.getCollection(collectionName);
+        slice = collection.getSlice(sliceId);
         if (slice.getLeader() != null && slice.getLeader().getState() == State.ACTIVE) {
           success = true;
           break;
@@ -889,15 +888,15 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
       }
 
       if (success) {
-        log.info("Successfully issued FORCELEADER command for collection: {}, shard: {}", collection, sliceId);
+        log.info("Successfully issued FORCELEADER command for collection: {}, shard: {}", collectionName, sliceId);
       } else {
-        log.info("Couldn't successfully force leader, collection: {}, shard: {}. Cluster state: {}", collection, sliceId, clusterState);
+        log.info("Couldn't successfully force leader, collection: {}, shard: {}. Cluster state: {}", collectionName, sliceId, clusterState);
       }
     } catch (SolrException e) {
       throw e;
     } catch (Exception e) {
       throw new SolrException(ErrorCode.SERVER_ERROR,
-          "Error executing FORCELEADER operation for collection: " + collection + " shard: " + sliceId, e);
+          "Error executing FORCELEADER operation for collection: " + collectionName + " shard: " + sliceId, e);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/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 e755b82..bdc9168 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
@@ -369,7 +369,7 @@ enum CoreAdminOperation {
           String collectionName = req.getCore().getCoreDescriptor().getCloudDescriptor().getCollectionName();
           DocCollection collection = clusterState.getCollection(collectionName);
           String sliceName = req.getCore().getCoreDescriptor().getCloudDescriptor().getShardId();
-          Slice slice = clusterState.getSlice(collectionName, sliceName);
+          Slice slice = collection.getSlice(sliceName);
           router = collection.getRouter() != null ? collection.getRouter() : DocRouter.DEFAULT;
           if (ranges == null) {
             DocRouter.Range currentRange = slice.getRange();
@@ -461,7 +461,7 @@ enum CoreAdminOperation {
             // to accept updates
             CloudDescriptor cloudDescriptor = core.getCoreDescriptor()
                 .getCloudDescriptor();
-            String collection = cloudDescriptor.getCollectionName();
+            String collectionName = cloudDescriptor.getCollectionName();
 
             if (retry % 15 == 0) {
               if (retry > 0 && log.isInfoEnabled())
@@ -471,7 +471,7 @@ enum CoreAdminOperation {
                     waitForState + "; forcing ClusterState update from ZooKeeper");
 
               // force a cluster state update
-              coreContainer.getZkController().getZkStateReader().forceUpdateCollection(collection);
+              coreContainer.getZkController().getZkStateReader().forceUpdateCollection(collectionName);
             }
 
             if (maxTries == 0) {
@@ -484,7 +484,8 @@ enum CoreAdminOperation {
             }
 
             ClusterState clusterState = coreContainer.getZkController().getClusterState();
-            Slice slice = clusterState.getSlice(collection, cloudDescriptor.getShardId());
+            DocCollection collection = clusterState.getCollection(collectionName);
+            Slice slice = collection.getSlice(cloudDescriptor.getShardId());
             if (slice != null) {
               final Replica replica = slice.getReplicasMap().get(coreNodeName);
               if (replica != null) {
@@ -508,7 +509,7 @@ enum CoreAdminOperation {
                 }
 
                 boolean onlyIfActiveCheckResult = onlyIfLeaderActive != null && onlyIfLeaderActive && localState != Replica.State.ACTIVE;
-                log.info("In WaitForState(" + waitForState + "): collection=" + collection + ", shard=" + slice.getName() +
+                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 +

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
index eb90762..6ba571a 100644
--- a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
+++ b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
@@ -66,6 +66,7 @@ import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.Aliases;
 import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkNodeProps;
@@ -756,11 +757,15 @@ public class HttpSolrCall {
     return result;
   }
 
-  private SolrCore getCoreByCollection(String collection) {
+  private SolrCore getCoreByCollection(String collectionName) {
     ZkStateReader zkStateReader = cores.getZkController().getZkStateReader();
 
     ClusterState clusterState = zkStateReader.getClusterState();
-    Map<String, Slice> slices = clusterState.getActiveSlicesMap(collection);
+    DocCollection collection = clusterState.getCollectionOrNull(collectionName);
+    if (collection == null) {
+      return null;
+    }
+    Map<String, Slice> slices = collection.getActiveSlicesMap();
     if (slices == null) {
       return null;
     }
@@ -773,7 +778,7 @@ public class HttpSolrCall {
     //For queries it doesn't matter and hence we don't distinguish here.
     for (Map.Entry<String, Slice> entry : entries) {
       // first see if we have the leader
-      Replica leaderProps = clusterState.getLeader(collection, entry.getKey());
+      Replica leaderProps = collection.getLeader(entry.getKey());
       if (leaderProps != null && liveNodes.contains(leaderProps.getNodeName()) && leaderProps.getState() == Replica.State.ACTIVE) {
         core = checkProps(leaderProps);
         if (core != null) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
index 5f4e4f1..67c88dd 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
@@ -547,8 +547,8 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
                       throw new SolrException(ErrorCode.SERVER_ERROR,
                           "No active slices serving " + id + " found for target collection: " + rule.getTargetCollectionName());
                     }
-                    Replica targetLeader = cstate.getLeader(rule.getTargetCollectionName(), activeSlices.iterator().next().getName());
-                    if (nodes == null) nodes = new ArrayList<>(1);
+                    Replica targetLeader = targetColl.getLeader(activeSlices.iterator().next().getName());
+                    nodes = new ArrayList<>(1);
                     nodes.add(new StdNode(new ZkCoreNodeProps(targetLeader)));
                     break;
                   }
@@ -596,7 +596,8 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
     ClusterState clusterState = zkController.getClusterState();
         
     CloudDescriptor cloudDescriptor = req.getCore().getCoreDescriptor().getCloudDescriptor();
-    Slice mySlice = clusterState.getSlice(collection, cloudDescriptor.getShardId());
+    DocCollection docCollection = clusterState.getCollection(collection);
+    Slice mySlice = docCollection.getSlice(cloudDescriptor.getShardId());
     boolean localIsLeader = cloudDescriptor.isLeader();
     if (DistribPhase.FROMLEADER == phase && localIsLeader && from != null) { // from will be null on log replay
       String fromShard = req.getParams().get(DISTRIB_FROM_PARENT);
@@ -606,7 +607,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
               "Request says it is coming from parent shard leader but we are in active state");
         }
         // shard splitting case -- check ranges to see if we are a sub-shard
-        Slice fromSlice = zkController.getClusterState().getCollection(collection).getSlice(fromShard);
+        Slice fromSlice = docCollection.getSlice(fromShard);
         DocRouter.Range parentRange = fromSlice.getRange();
         if (parentRange == null) parentRange = new DocRouter.Range(Integer.MIN_VALUE, Integer.MAX_VALUE);
         if (mySlice.getRange() != null && !mySlice.getRange().isSubsetOf(parentRange)) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/util/SolrCLI.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/SolrCLI.java b/solr/core/src/java/org/apache/solr/util/SolrCLI.java
index 9627671..d21400d 100644
--- a/solr/core/src/java/org/apache/solr/util/SolrCLI.java
+++ b/solr/core/src/java/org/apache/solr/util/SolrCLI.java
@@ -88,6 +88,7 @@ import org.apache.solr.client.solrj.request.ContentStreamUpdateRequest;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkCoreNodeProps;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/util/SolrLogLayout.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/SolrLogLayout.java b/solr/core/src/java/org/apache/solr/util/SolrLogLayout.java
index 8b1ca1f..b79ec0c 100644
--- a/solr/core/src/java/org/apache/solr/util/SolrLogLayout.java
+++ b/solr/core/src/java/org/apache/solr/util/SolrLogLayout.java
@@ -28,6 +28,7 @@ import org.apache.log4j.spi.ThrowableInformation;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.StringUtils;
+import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.util.SuppressForbidden;
 import org.apache.solr.core.SolrCore;
@@ -236,10 +237,11 @@ public class SolrLogLayout extends Layout {
     return sb.toString();
   }
 
-  private Map<String,Object> getReplicaProps(ZkController zkController, SolrCore core) {
-    final String collection = core.getCoreDescriptor().getCloudDescriptor().getCollectionName();
-    Replica replica = zkController.getClusterState().getReplica(collection, zkController.getCoreNodeName(core.getCoreDescriptor()));
-    if(replica!=null) {
+  private Map<String, Object> getReplicaProps(ZkController zkController, SolrCore core) {
+    final String collectionName = core.getCoreDescriptor().getCloudDescriptor().getCollectionName();
+    DocCollection collection = zkController.getClusterState().getCollectionOrNull(collectionName);
+    Replica replica = collection.getReplica(zkController.getCoreNodeName(core.getCoreDescriptor()));
+    if (replica != null) {
       return replica.getProperties();
     }
     return Collections.EMPTY_MAP;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/test/org/apache/solr/cloud/AssignTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/AssignTest.java b/solr/core/src/test/org/apache/solr/cloud/AssignTest.java
index 6069650..7593f3b 100644
--- a/solr/core/src/test/org/apache/solr/cloud/AssignTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/AssignTest.java
@@ -82,7 +82,7 @@ public class AssignTest extends SolrTestCaseJ4 {
     
     Set<String> liveNodes = new HashSet<>();
     ClusterState state = new ClusterState(-1,liveNodes, collectionStates);
-    String nodeName = Assign.assignNode("collection1", state);
+    String nodeName = Assign.assignNode(state.getCollection("collection1"));
     
     assertEquals("core_node2", nodeName);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java
index accc36a..7dd77d9 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java
@@ -1193,12 +1193,12 @@ public class CollectionsAPIDistributedZkTest extends AbstractFullDistribZkTestBa
     String collectionName = "addReplicaColl";
     try (CloudSolrClient client = createCloudClient(null)) {
       createCollection(collectionName, client, 2, 2);
-      String newReplicaName = Assign.assignNode(collectionName, client.getZkStateReader().getClusterState());
+      String newReplicaName = Assign.assignNode(client.getZkStateReader().getClusterState().getCollection(collectionName));
       ArrayList<String> nodeList = new ArrayList<>(client.getZkStateReader().getClusterState().getLiveNodes());
       Collections.shuffle(nodeList, random());
 
       Replica newReplica = doAddReplica(collectionName, "shard1",
-          Assign.assignNode(collectionName, client.getZkStateReader().getClusterState()),
+          Assign.assignNode(client.getZkStateReader().getClusterState().getCollection(collectionName)),
           nodeList.get(0), client, null);
 
       log.info("newReplica {},\n{} ", newReplica, client.getZkStateReader().getBaseUrlForNodeName(nodeList.get(0)));
@@ -1210,7 +1210,7 @@ public class CollectionsAPIDistributedZkTest extends AbstractFullDistribZkTestBa
       String instancePathStr = createTempDir().toString();
       props.put(CoreAdminParams.INSTANCE_DIR, instancePathStr); //Use name via the property.instanceDir method
       newReplica = doAddReplica(collectionName, "shard2",
-          Assign.assignNode(collectionName, client.getZkStateReader().getClusterState()),
+          Assign.assignNode(client.getZkStateReader().getClusterState().getCollection(collectionName)),
           null, client, props);
       assertNotNull(newReplica);
 
@@ -1244,7 +1244,7 @@ public class CollectionsAPIDistributedZkTest extends AbstractFullDistribZkTestBa
       props.put(CoreAdminParams.NAME, "propertyDotName");
 
       newReplica = doAddReplica(collectionName, "shard1",
-          Assign.assignNode(collectionName, client.getZkStateReader().getClusterState()),
+          Assign.assignNode(client.getZkStateReader().getClusterState().getCollection(collectionName)),
           nodeList.get(0), client, props);
       assertEquals("'core' should be 'propertyDotName' ", "propertyDotName", newReplica.getStr("core"));
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
index 52ae96f..0975b9a 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
@@ -276,7 +276,7 @@ public class CollectionsAPISolrJTest extends AbstractFullDistribZkTestBase {
 
     cloudClient.setDefaultCollection(collectionName);
 
-    String newReplicaName = Assign.assignNode(collectionName, cloudClient.getZkStateReader().getClusterState());
+    String newReplicaName = Assign.assignNode(cloudClient.getZkStateReader().getClusterState().getCollection(collectionName));
     ArrayList<String> nodeList = new ArrayList<>(cloudClient.getZkStateReader().getClusterState().getLiveNodes());
     Collections.shuffle(nodeList, random());
     CollectionAdminRequest.AddReplica addReplica = new CollectionAdminRequest.AddReplica()

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/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 85a88ec..7863899 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
@@ -25,6 +25,7 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
@@ -687,7 +688,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
       while (version == getClusterStateVersion(zkClient));
       Thread.sleep(500);
       assertTrue(collection+" should remain after removal of the last core", // as of SOLR-5209 core removal does not cascade to remove the slice and collection
-          reader.getClusterState().getCollections().contains(collection));
+          reader.getClusterState().hasCollection(collection));
       assertTrue(core_node+" should be gone after publishing the null state",
           null == reader.getClusterState().getCollection(collection).getReplica(core_node));
     } finally {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/test/org/apache/solr/handler/TestSQLHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/TestSQLHandler.java b/solr/core/src/test/org/apache/solr/handler/TestSQLHandler.java
index 9eca6db..893b6fe 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestSQLHandler.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestSQLHandler.java
@@ -30,6 +30,7 @@ import org.apache.solr.client.solrj.io.stream.ExceptionStream;
 import org.apache.solr.client.solrj.io.stream.SolrStream;
 import org.apache.solr.client.solrj.io.stream.TupleStream;
 import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
+import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.params.CommonParams;
 import org.junit.After;
 import org.junit.AfterClass;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/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 1a6139e..97a804d 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
@@ -51,6 +51,7 @@ 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;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/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 2495c41..d60cccb 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
@@ -90,7 +90,9 @@ public class ClusterState implements JSONWriter.Writable {
 
   /**
    * Get the lead replica for specific collection, or null if one currently doesn't exist.
+   * @deprecated Use {@link DocCollection#getLeader(String)} instead
    */
+  @Deprecated
   public Replica getLeader(String collection, String sliceName) {
     DocCollection coll = getCollectionOrNull(collection);
     if (coll == null) return null;
@@ -98,14 +100,6 @@ public class ClusterState implements JSONWriter.Writable {
     if (slice == null) return null;
     return slice.getLeader();
   }
-  private Replica getReplica(DocCollection coll, String replicaName) {
-    if (coll == null) return null;
-    for (Slice slice : coll.getSlices()) {
-      Replica replica = slice.getReplica(replicaName);
-      if (replica != null) return replica;
-    }
-    return null;
-  }
 
   /**
    * Returns true if the specified collection name exists, false otherwise.
@@ -113,48 +107,76 @@ public class ClusterState implements JSONWriter.Writable {
    * Implementation note: This method resolves the collection reference by calling
    * {@link CollectionRef#get()} which can make a call to ZooKeeper. This is necessary
    * because the semantics of how collection list is loaded have changed in SOLR-6629.
-   * Please javadocs in {@link ZkStateReader#refreshCollectionList(Watcher)}
+   * Please see javadocs in {@link ZkStateReader#refreshCollectionList(Watcher)}
    */
   public boolean hasCollection(String collectionName) {
     return getCollectionOrNull(collectionName) != null;
   }
 
   /**
-   * Gets the replica by the core name (assuming the slice is unknown) or null if replica is not found.
+   * Gets the replica by the core node name (assuming the slice is unknown) or null if replica is not found.
    * If the slice is known, do not use this method.
    * coreNodeName is the same as replicaName
+   *
+   * @deprecated use {@link DocCollection#getReplica(String)} instead
    */
+  @Deprecated
   public Replica getReplica(final String collection, final String coreNodeName) {
-    return getReplica(getCollectionOrNull(collection), coreNodeName);
+    DocCollection coll = getCollectionOrNull(collection);
+    if (coll == null) return null;
+    for (Slice slice : coll.getSlices()) {
+      Replica replica = slice.getReplica(coreNodeName);
+      if (replica != null) return replica;
+    }
+    return null;
   }
 
   /**
    * Get the named Slice for collection, or null if not found.
+   *
+   * @deprecated use {@link DocCollection#getSlice(String)} instead
    */
+  @Deprecated
   public Slice getSlice(String collection, String sliceName) {
     DocCollection coll = getCollectionOrNull(collection);
     if (coll == null) return null;
     return coll.getSlice(sliceName);
   }
 
+  /**
+   * @deprecated use {@link DocCollection#getSlicesMap()} instead
+   */
+  @Deprecated
   public Map<String, Slice> getSlicesMap(String collection) {
     DocCollection coll = getCollectionOrNull(collection);
     if (coll == null) return null;
     return coll.getSlicesMap();
   }
-  
+
+  /**
+   * @deprecated use {@link DocCollection#getActiveSlicesMap()} instead
+   */
+  @Deprecated
   public Map<String, Slice> getActiveSlicesMap(String collection) {
     DocCollection coll = getCollectionOrNull(collection);
     if (coll == null) return null;
     return coll.getActiveSlicesMap();
   }
 
+  /**
+   * @deprecated use {@link DocCollection#getSlices()} instead
+   */
+  @Deprecated
   public Collection<Slice> getSlices(String collection) {
     DocCollection coll = getCollectionOrNull(collection);
     if (coll == null) return null;
     return coll.getSlices();
   }
 
+  /**
+   * @deprecated use {@link DocCollection#getActiveSlices()} instead
+   */
+  @Deprecated
   public Collection<Slice> getActiveSlices(String collection) {
     DocCollection coll = getCollectionOrNull(collection);
     if (coll == null) return null;
@@ -195,7 +217,7 @@ public class ClusterState implements JSONWriter.Writable {
    * Implementation note: This method resolves the collection reference by calling
    * {@link CollectionRef#get()} which can make a call to ZooKeeper. This is necessary
    * because the semantics of how collection list is loaded have changed in SOLR-6629.
-   * Please javadocs in {@link ZkStateReader#refreshCollectionList(Watcher)}
+   * Please see javadocs in {@link ZkStateReader#refreshCollectionList(Watcher)}
    */
   public Set<String> getCollections() {
     Set<String> result = new HashSet<>();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterStateUtil.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterStateUtil.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterStateUtil.java
index 1628756..a298cb3 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterStateUtil.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterStateUtil.java
@@ -20,6 +20,7 @@ import java.lang.invoke.MethodHandles;
 
 import java.util.Collection;
 import java.util.Collections;
+import java.util.List;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java b/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java
index e8f26e1..9e4418c 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java
@@ -209,4 +209,10 @@ public class DocCollection extends ZkNodeProps {
     }
     return null;
   }
+
+  public Replica getLeader(String sliceName) {
+    Slice slice = getSlice(sliceName);
+    if (slice == null) return null;
+    return slice.getLeader();
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/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 c6f88c0..568c791 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
@@ -646,7 +646,8 @@ public class ZkStateReader implements Closeable {
 
   public Replica getLeader(String collection, String shard) {
     if (clusterState != null) {
-      Replica replica = clusterState.getLeader(collection, shard);
+      DocCollection docCollection = clusterState.getCollectionOrNull(collection);
+      Replica replica = docCollection != null ? docCollection.getLeader(shard) : null;
       if (replica != null && getClusterState().liveNodesContain(replica.getNodeName())) {
         return replica;
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/solrj/src/test/org/apache/solr/client/solrj/io/sql/JdbcTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/sql/JdbcTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/sql/JdbcTest.java
index 9050092..d38661e 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/sql/JdbcTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/sql/JdbcTest.java
@@ -34,6 +34,7 @@ import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.LuceneTestCase.Slow;
 import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
 import org.apache.solr.cloud.AbstractZkTestCase;
+import org.apache.solr.common.cloud.DocCollection;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;