You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2016/02/25 19:41:54 UTC

[01/18] lucene-solr git commit: a rough start

Repository: lucene-solr
Updated Branches:
  refs/heads/master e901c6197 -> b9d46e4fd


a rough start


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

Branch: refs/heads/master
Commit: 013abeaa5b7b1a817f5e94016fccaee93fafd732
Parents: 18bb8ca
Author: Mike McCandless <mi...@apache.org>
Authored: Tue Feb 23 16:34:39 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Tue Feb 23 16:34:39 2016 -0500

----------------------------------------------------------------------
 .../apache/lucene/index/PrefixCodedTerms.java   |   2 +-
 .../apache/lucene/index/TermsHashPerField.java  |   3 +
 .../apache/lucene/search/PointInSetQuery.java   | 341 +++++++++++++++++++
 .../apache/lucene/search/TestPointQueries.java  |  35 ++
 4 files changed, 380 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/013abeaa/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java b/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java
index 87a9ae2..3dca3db 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java
@@ -67,7 +67,7 @@ public class PrefixCodedTerms implements Accountable {
       add(term.field(), term.bytes());
     }
 
-    /** add a term */
+    /** add a term.  This fully consumes in the incoming {@link BytesRef}. */
     public void add(String field, BytesRef bytes) {
       assert lastTerm.equals(new Term("")) || new Term(field, bytes).compareTo(lastTerm) > 0;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/013abeaa/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java b/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java
index 3275a4e..b06295c 100644
--- a/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java
+++ b/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java
@@ -93,7 +93,10 @@ abstract class TermsHashPerField implements Comparable<TermsHashPerField> {
   /** Collapse the hash table and sort in-place; also sets
    * this.sortedTermIDs to the results */
   public int[] sortPostings() {
+    long t0 = System.nanoTime();
     sortedTermIDs = bytesHash.sort(BytesRef.getUTF8SortedAsUnicodeComparator());
+    // nocommit
+    System.out.println("MKM: field " + fieldInfo.name + " has " + bytesHash.size() + " unique terms " + ((System.nanoTime()-t0)/1000000.0) + " msec to sort");
     return sortedTermIDs;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/013abeaa/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java b/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
new file mode 100644
index 0000000..4403e45
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
@@ -0,0 +1,341 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search;
+
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Objects;
+
+import org.apache.lucene.document.IntPoint;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.PointValues.IntersectVisitor;
+import org.apache.lucene.index.PointValues.Relation;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.index.PrefixCodedTerms.TermIterator;
+import org.apache.lucene.index.PrefixCodedTerms;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.BytesRefIterator;
+import org.apache.lucene.util.DocIdSetBuilder;
+import org.apache.lucene.util.NumericUtils;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.StringHelper;
+
+/** Finds all documents whose point value, previously indexed with e.g. {@link org.apache.lucene.document.LongPoint}, is contained in the
+ *  specified set */
+
+// nocommit make abstract
+public class PointInSetQuery extends Query {
+  // A little bit overkill for us, since all of our "terms" are always in the same field:
+  final PrefixCodedTerms sortedPackedPoints;
+  final int sortedPackedPointsHashCode;
+  final String field;
+  final int numDims;
+  final int bytesPerDim;
+
+  /** {@code packedPoints} must already be sorted! */
+  protected PointInSetQuery(String field, int numDims, int bytesPerDim, BytesRefIterator packedPoints) throws IOException {
+    this.field = field;
+    // nocommit validate these:
+    this.bytesPerDim = bytesPerDim;
+    this.numDims = numDims;
+
+    // In the 1D case this works well (the more points, the more common prefixes they share, typically), but in
+    // the > 1 D case, where we are only looking at the first dimension's prefix bytes, it can at worst not hurt:
+    PrefixCodedTerms.Builder builder = new PrefixCodedTerms.Builder();
+    BytesRefBuilder previous = null;
+    BytesRef current;
+    while ((current = packedPoints.next()) != null) {
+      if (current.length != numDims * bytesPerDim) {
+        throw new IllegalArgumentException("packed point length should be " + (numDims * bytesPerDim) + " but got " + current.length + "; field=\"" + field + "\", numDims=" + numDims + " bytesPerDim=" + bytesPerDim);
+      }
+      if (previous == null) {
+        previous = new BytesRefBuilder();
+      } else if (previous.get().equals(current)) {
+        continue; // deduplicate
+      }
+      builder.add(field, current);
+      previous.copyBytes(current);
+    }
+    sortedPackedPoints = builder.finish();
+    sortedPackedPointsHashCode = sortedPackedPoints.hashCode();
+  }
+
+  /** Use in the 1D case when you indexed 1D int values using {@link org.apache.lucene.document.IntPoint} */
+  public static PointInSetQuery newIntSet(String field, int... valuesIn) {
+
+    // Don't unexpectedly change the user's incoming array:
+    int[] values = valuesIn.clone();
+
+    Arrays.sort(values);
+
+    final BytesRef value = new BytesRef(new byte[Integer.BYTES]);
+    value.length = Integer.BYTES;
+
+    try {
+      return new PointInSetQuery(field, 1, Integer.BYTES,
+                                 new BytesRefIterator() {
+
+                                   int upto;
+
+                                   @Override
+                                   public BytesRef next() {
+                                     if (upto == values.length) {
+                                       return null;
+                                     } else {
+                                       IntPoint.encodeDimension(values[upto], value.bytes, 0);
+                                       upto++;
+                                       return value;
+                                     }
+                                   }
+                                 });
+    } catch (IOException bogus) {
+      // Should never happen ;)
+      throw new RuntimeException(bogus);
+    }
+  }
+
+  @Override
+  public 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) {
+
+      @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);
+        }
+        int bytesPerDim = fieldInfo.getPointNumBytes();
+
+        DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc());
+
+        int[] hitCount = new int[1];
+        final TermIterator iterator = sortedPackedPoints.iterator();
+        byte[] pointBytes = new byte[bytesPerDim * numDims];
+
+        if (numDims == 1) {
+
+          final BytesRef scratch = new BytesRef();
+          scratch.length = bytesPerDim;
+
+          // Optimize this common case, effectively doing a merge sort of the indexed values vs the queried set:
+          values.intersect(field,
+                           new IntersectVisitor() {
+
+                             private BytesRef nextQueryPoint = iterator.next();
+
+                             @Override
+                             public void grow(int count) {
+                               result.grow(count);
+                             }
+
+                             @Override
+                             public void visit(int docID) {
+                               hitCount[0]++;
+                               result.add(docID);
+                             }
+
+                             @Override
+                             public void visit(int docID, byte[] packedValue) {
+                               scratch.bytes = packedValue;
+                               while (nextQueryPoint != null) {
+                                 int cmp = nextQueryPoint.compareTo(scratch);
+                                 if (cmp == 0) {
+                                   // Query point equals index point, so collect and return
+                                   hitCount[0]++;
+                                   result.add(docID);
+                                   break;
+                                 } else if (cmp < 0) {
+                                   // Query point is before index point, so we move to next query point
+                                   nextQueryPoint = iterator.next();
+                                 } else {
+                                   // Query point is after index point, so we don't collect and we return:
+                                   break;
+                                 }
+                               }
+                             }
+
+                             @Override
+                             public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+
+                               while (nextQueryPoint != null) {
+                                 scratch.bytes = minPackedValue;
+                                 int cmpMin = nextQueryPoint.compareTo(scratch);
+                                 if (cmpMin < 0) {
+                                   // query point is before the start of this cell
+                                   nextQueryPoint = iterator.next();
+                                   continue;
+                                 }
+                                 scratch.bytes = maxPackedValue;
+                                 int cmpMax = nextQueryPoint.compareTo(scratch);
+                                 if (cmpMax > 0) {
+                                   // query point is after the end of this cell
+                                   return Relation.CELL_OUTSIDE_QUERY;
+                                 }
+
+                                 if (cmpMin == 0 && cmpMax == 0) {
+                                   // NOTE: we only hit this if we are on a cell whose min and max values are exactly equal to our point,
+                                   // which can easily happen if many (> 1024) docs share this one value
+                                   return Relation.CELL_INSIDE_QUERY;
+                                 } else {
+                                   return Relation.CELL_CROSSES_QUERY;
+                                 }
+                               }
+
+                               // We exhausted all points in the query:
+                               return Relation.CELL_OUTSIDE_QUERY;
+                             }
+                           });
+        } else {
+          for (BytesRef point = iterator.next(); point != null; point = iterator.next()) {
+            // nocommit make sure a test tests this:
+            assert point.length == pointBytes.length;
+            System.arraycopy(point.bytes, point.offset, pointBytes, 0, pointBytes.length);
+
+            final BytesRef finalPoint = point;
+
+            values.intersect(field,
+                             // nocommit don't make new instance of this for each point?
+                             new IntersectVisitor() {
+
+                               @Override
+                               public void grow(int count) {
+                                 result.grow(count);
+                               }
+
+                               @Override
+                               public void visit(int docID) {
+                                 hitCount[0]++;
+                                 result.add(docID);
+                               }
+
+                               @Override
+                               public void visit(int docID, byte[] packedValue) {
+                                 assert packedValue.length == finalPoint.length;
+                                 if (Arrays.equals(packedValue, pointBytes)) {
+                                   // The point for this doc matches the point we are querying on
+                                   hitCount[0]++;
+                                   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;
+
+                                   int cmpMin = StringHelper.compare(bytesPerDim, minPackedValue, offset, pointBytes, offset);
+                                   if (cmpMin > 0) {
+                                     return Relation.CELL_OUTSIDE_QUERY;
+                                   }
+
+                                   int cmpMax = StringHelper.compare(bytesPerDim, maxPackedValue, offset, pointBytes, offset);
+                                   if (cmpMax < 0) {
+                                     return Relation.CELL_OUTSIDE_QUERY;
+                                   }
+
+                                   if (cmpMin != 0 || cmpMax != 0) {
+                                     crosses = true;
+                                   }
+                                 }
+
+                                 if (crosses) {
+                                   return Relation.CELL_CROSSES_QUERY;
+                                 } else {
+                                   // nocommit make sure tests hit this case:
+                                   // NOTE: we only hit this if we are on a cell whose min and max values are exactly equal to our point,
+                                   // which can easily happen if many docs share this one value
+                                   return Relation.CELL_INSIDE_QUERY;
+                                 }
+                               }
+                             });
+          }
+        }
+
+        // NOTE: hitCount[0] will be over-estimate in multi-valued case
+        return new ConstantScoreScorer(this, score(), result.build(hitCount[0]).iterator());
+      }
+    };
+  }
+
+  @Override
+  public int hashCode() {
+    int hash = super.hashCode();
+    hash += sortedPackedPointsHashCode^0x14fa55fb;
+    hash += numDims^0x14fa55fb;
+    hash += bytesPerDim^0x14fa55fb;
+    return hash;
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (super.equals(other)) {
+      final PointInSetQuery q = (PointInSetQuery) other;
+      return q.numDims == numDims &&
+        q.bytesPerDim == bytesPerDim &&
+        q.sortedPackedPointsHashCode == sortedPackedPointsHashCode &&
+        q.sortedPackedPoints.equals(sortedPackedPoints);
+    }
+
+    return false;
+  }
+
+  @Override
+  public String toString(String field) {
+    final StringBuilder sb = new StringBuilder();
+    sb.append(getClass().getSimpleName());
+    sb.append(':');
+    if (this.field.equals(field) == false) {
+      sb.append("field=");
+      sb.append(this.field);
+      sb.append(':');
+    }
+
+    TermIterator iterator = sortedPackedPoints.iterator();
+    for (BytesRef point = iterator.next(); point != null; point = iterator.next()) {
+      sb.append(' ');
+      // nocommit fix me to convert back to the numbers/etc.:
+      sb.append(point);
+    }
+
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/013abeaa/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 cfab5fb..ea432f7 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
@@ -1114,4 +1114,39 @@ public class TestPointQueries extends LuceneTestCase {
                                                                       new boolean[] { false, true }).toString());
 
   }
+
+  // nocommit fix existing randomized tests to sometimes randomly use PointInSet instead
+
+  // nocommit need 2D test too
+
+  public void testBasicPointInSetQuery() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = newIndexWriterConfig();
+    iwc.setCodec(getCodec());
+    IndexWriter w = new IndexWriter(dir, iwc);
+
+    Document doc = new Document();
+    doc.add(new IntPoint("int", 17));
+    w.addDocument(doc);
+
+    doc = new Document();
+    doc.add(new IntPoint("int", 42));
+    w.addDocument(doc);
+
+    doc = new Document();
+    doc.add(new IntPoint("int", 97));
+    w.addDocument(doc);
+
+    IndexReader r = DirectoryReader.open(w);
+    IndexSearcher s = newSearcher(r);
+    assertEquals(0, s.count(PointInSetQuery.newIntSet("int", 16)));
+    assertEquals(1, s.count(PointInSetQuery.newIntSet("int", 17)));
+    assertEquals(3, s.count(PointInSetQuery.newIntSet("int", 17, 97, 42)));
+    assertEquals(3, s.count(PointInSetQuery.newIntSet("int", -7, 17, 42, 97)));
+    assertEquals(3, s.count(PointInSetQuery.newIntSet("int", 17, 20, 42, 97)));
+    assertEquals(3, s.count(PointInSetQuery.newIntSet("int", 17, 105, 42, 97)));
+    w.close();
+    r.close();
+    dir.close();
+  }
 }


[11/18] lucene-solr git commit: Merge branch 'master' into point_set_query

Posted by mi...@apache.org.
Merge branch 'master' into point_set_query


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

Branch: refs/heads/master
Commit: 1c3d52d6b54a024ce8b577fb04ecca9e78d3c07f
Parents: 1fb9de8 dc95d52
Author: Mike McCandless <mi...@apache.org>
Authored: Wed Feb 24 15:38:40 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Wed Feb 24 15:38:40 2016 -0500

----------------------------------------------------------------------
 .../org/apache/lucene/document/DoublePoint.java | 10 +--
 .../org/apache/lucene/document/FieldType.java   | 14 ++--
 .../org/apache/lucene/document/FloatPoint.java  | 10 +--
 .../org/apache/lucene/document/IntPoint.java    | 10 +--
 .../org/apache/lucene/document/LongPoint.java   | 10 +--
 .../org/apache/lucene/document/TestField.java   | 16 ++---
 .../apache/lucene/document/TestFieldType.java   |  6 ++
 .../apache/lucene/document/BigIntegerPoint.java |  4 +-
 .../lucene/document/InetAddressPoint.java       |  4 +-
 .../org/apache/lucene/document/LatLonPoint.java |  4 +-
 .../lucene/document/TestBigIntegerPoint.java    |  4 +-
 .../lucene/document/TestInetAddressPoint.java   |  6 +-
 .../apache/lucene/document/TestLatLonPoint.java |  2 +-
 .../org/apache/lucene/geo3d/Geo3DPoint.java     | 67 +++++++++++++++++---
 .../lucene/geo3d/PointInGeo3DShapeQuery.java    | 10 +--
 .../org/apache/lucene/geo3d/TestGeo3DPoint.java | 18 ++++--
 16 files changed, 132 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1c3d52d6/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1c3d52d6/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1c3d52d6/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1c3d52d6/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1c3d52d6/lucene/sandbox/src/java/org/apache/lucene/document/BigIntegerPoint.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1c3d52d6/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressPoint.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1c3d52d6/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1c3d52d6/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
----------------------------------------------------------------------


[17/18] lucene-solr git commit: LUCENE-7048: add XXXPoint.newSetQuery to match documents with any values from the specified set (this is the analog of TermsQuery, for points)

Posted by mi...@apache.org.
LUCENE-7048: add XXXPoint.newSetQuery to match documents with any values from the specified set (this is the analog of TermsQuery, for points)

Merge branch 'point_set_query'


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

Branch: refs/heads/master
Commit: 446ce8604e4baff4f4e486e39f7e885f0d8d0c57
Parents: e901c61 7e1fbdf
Author: Mike McCandless <mi...@apache.org>
Authored: Thu Feb 25 13:08:35 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Thu Feb 25 13:08:35 2016 -0500

----------------------------------------------------------------------
 .../org/apache/lucene/document/BinaryPoint.java |  68 ++
 .../org/apache/lucene/document/DoublePoint.java |  49 +-
 .../org/apache/lucene/document/FloatPoint.java  |  49 +-
 .../org/apache/lucene/document/IntPoint.java    |  45 ++
 .../org/apache/lucene/document/LongPoint.java   |  45 ++
 .../apache/lucene/index/PrefixCodedTerms.java   |   2 +-
 .../apache/lucene/search/PointInSetQuery.java   | 370 ++++++++++
 .../org/apache/lucene/util/NumericUtils.java    |  15 +-
 .../apache/lucene/search/TestPointQueries.java  | 711 +++++++++++++++++++
 .../apache/lucene/document/BigIntegerPoint.java |  44 ++
 .../lucene/document/InetAddressPoint.java       |  49 +-
 .../org/apache/lucene/document/LatLonPoint.java |   2 +-
 .../lucene/document/TestBigIntegerPoint.java    |  11 +
 .../lucene/document/TestInetAddressPoint.java   |   4 +
 14 files changed, 1444 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/446ce860/lucene/core/src/java/org/apache/lucene/document/BinaryPoint.java
----------------------------------------------------------------------
diff --cc lucene/core/src/java/org/apache/lucene/document/BinaryPoint.java
index 201001a,9c4de99..b8b53d5
--- a/lucene/core/src/java/org/apache/lucene/document/BinaryPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/BinaryPoint.java
@@@ -19,12 -25,13 +25,14 @@@ import org.apache.lucene.search.PointIn
  import org.apache.lucene.search.PointRangeQuery;
  import org.apache.lucene.search.Query;
  import org.apache.lucene.util.BytesRef;
+ import org.apache.lucene.util.BytesRefIterator;
+ import org.apache.lucene.util.StringHelper;
  
  /** 
 - * A binary field that is indexed dimensionally such that finding
 - * all documents within an N-dimensional shape or range at search time is
 - * efficient.  Multiple values for the same field in one documents
 + * An indexed binary field.
 + * <p>
 + * Finding all documents within an N-dimensional shape or range at search time is
 + * efficient.  Multiple values for the same field in one document
   * is allowed.
   * <p>
   * This field defines static factory methods for creating common queries:

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/446ce860/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/446ce860/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/446ce860/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/446ce860/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/446ce860/lucene/sandbox/src/java/org/apache/lucene/document/BigIntegerPoint.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/446ce860/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressPoint.java
----------------------------------------------------------------------
diff --cc lucene/sandbox/src/java/org/apache/lucene/document/InetAddressPoint.java
index 4e5d699,3081875..1a73dc1
--- a/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressPoint.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressPoint.java
@@@ -22,11 -25,11 +25,12 @@@ import org.apache.lucene.search.PointIn
  import org.apache.lucene.search.PointRangeQuery;
  import org.apache.lucene.search.Query;
  import org.apache.lucene.util.BytesRef;
+ import org.apache.lucene.util.BytesRefIterator;
  
  /** 
 - * A field indexing {@link InetAddress} dimensionally such that finding
 - * all documents within a range at search time is
 + * An indexed 128-bit {@code InetAddress} field.
 + * <p>
 + * Finding all documents within a range at search time is
   * efficient.  Multiple values for the same field in one document
   * is allowed. 
   * <p>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/446ce860/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java
----------------------------------------------------------------------


[13/18] lucene-solr git commit: improve tests; fix a couple nocommits

Posted by mi...@apache.org.
improve tests; fix a couple nocommits


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

Branch: refs/heads/master
Commit: c056f4b3885ded22e4780ea9c61f95c52a3b654b
Parents: ae70920
Author: Mike McCandless <mi...@apache.org>
Authored: Wed Feb 24 16:59:01 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Wed Feb 24 16:59:01 2016 -0500

----------------------------------------------------------------------
 .../apache/lucene/search/PointInSetQuery.java   | 14 +++--
 .../apache/lucene/search/TestPointQueries.java  | 60 +++++++++++++++++++-
 2 files changed, 67 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c056f4b3/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java b/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
index 4da1c3e..c00973e 100644
--- a/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
@@ -42,7 +42,6 @@ import org.apache.lucene.util.StringHelper;
 /** Finds all documents whose point value, previously indexed with e.g. {@link org.apache.lucene.document.LongPoint}, is contained in the
  *  specified set */
 
-// nocommit explain that the 1D case must be pre-sorted
 public class PointInSetQuery extends Query {
   // A little bit overkill for us, since all of our "terms" are always in the same field:
   final PrefixCodedTerms sortedPackedPoints;
@@ -51,7 +50,7 @@ public class PointInSetQuery extends Query {
   final int numDims;
   final int bytesPerDim;
 
-  /** {@code packedPoints} must already be sorted! */
+  /** In the 1D case, the {@code packedPoints} iterator must be in sorted order. */
   protected PointInSetQuery(String field, int numDims, int bytesPerDim, BytesRefIterator packedPoints) throws IOException {
     this.field = field;
     if (bytesPerDim < 1 || bytesPerDim > PointValues.MAX_NUM_BYTES) {
@@ -61,6 +60,7 @@ public class PointInSetQuery extends Query {
     if (numDims < 1 || numDims > PointValues.MAX_DIMENSIONS) {
       throw new IllegalArgumentException("numDims must be > 0 and <= " + PointValues.MAX_DIMENSIONS + "; got " + numDims);
     }
+
     this.numDims = numDims;
 
     // In the 1D case this works well (the more points, the more common prefixes they share, typically), but in
@@ -74,9 +74,13 @@ public class PointInSetQuery extends Query {
       }
       if (previous == null) {
         previous = new BytesRefBuilder();
-      // nocommit detect out-of-order 1D case
-      } else if (previous.get().equals(current)) {
-        continue; // deduplicate
+      } else {
+        int cmp = previous.get().compareTo(current);
+        if (cmp == 0) {
+          continue; // deduplicate
+        } else if (numDims == 1 && cmp > 0) {
+          throw new IllegalArgumentException("numDims=1 and values are out of order: saw " + previous + " before " + current);
+        }
       }
       builder.add(field, current);
       previous.copyBytes(current);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c056f4b3/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 f9e6314..51d914d 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
@@ -96,6 +96,33 @@ public class TestPointQueries extends LuceneTestCase {
     }
   }
 
+  public void testBasicInts() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(new MockAnalyzer(random())));
+
+    Document doc = new Document();
+    doc.add(new IntPoint("point", -7));
+    w.addDocument(doc);
+
+    doc = new Document();
+    doc.add(new IntPoint("point", 0));
+    w.addDocument(doc);
+
+    doc = new Document();
+    doc.add(new IntPoint("point", 3));
+    w.addDocument(doc);
+
+    DirectoryReader r = DirectoryReader.open(w);
+    IndexSearcher s = new IndexSearcher(r);
+    assertEquals(2, s.count(IntPoint.newRangeQuery("point", -8, false, 1, false)));
+    assertEquals(3, s.count(IntPoint.newRangeQuery("point", -7, true, 3, true)));
+    assertEquals(1, s.count(IntPoint.newExactQuery("point", -7)));
+    assertEquals(0, s.count(IntPoint.newExactQuery("point", -6)));
+    w.close();
+    r.close();
+    dir.close();
+  }
+
   public void testBasicFloats() throws Exception {
     Directory dir = newDirectory();
     IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(new MockAnalyzer(random())));
@@ -123,6 +150,33 @@ public class TestPointQueries extends LuceneTestCase {
     dir.close();
   }
 
+  public void testBasicLongs() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(new MockAnalyzer(random())));
+
+    Document doc = new Document();
+    doc.add(new LongPoint("point", -7));
+    w.addDocument(doc);
+
+    doc = new Document();
+    doc.add(new LongPoint("point", 0));
+    w.addDocument(doc);
+
+    doc = new Document();
+    doc.add(new LongPoint("point", 3));
+    w.addDocument(doc);
+
+    DirectoryReader r = DirectoryReader.open(w);
+    IndexSearcher s = new IndexSearcher(r);
+    assertEquals(2, s.count(LongPoint.newRangeQuery("point", -8L, false, 1L, false)));
+    assertEquals(3, s.count(LongPoint.newRangeQuery("point", -7L, true, 3L, true)));
+    assertEquals(1, s.count(LongPoint.newExactQuery("point", -7L)));
+    assertEquals(0, s.count(LongPoint.newExactQuery("point", -6L)));
+    w.close();
+    r.close();
+    dir.close();
+  }
+
   public void testBasicDoubles() throws Exception {
     Directory dir = newDirectory();
     IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(new MockAnalyzer(random())));
@@ -1194,16 +1248,18 @@ public class TestPointQueries extends LuceneTestCase {
     boolean useNarrowRange = random().nextBoolean();
     final Integer valueMin;
     final Integer valueMax;
+    int numValues;
     if (useNarrowRange) {
       int gap = random().nextInt(100);
       valueMin = random().nextInt(Integer.MAX_VALUE-gap);
       valueMax = valueMin + gap;
+      numValues = TestUtil.nextInt(random(), 1, gap+1);
     } else {
       valueMin = null;
       valueMax = null;
+      numValues = TestUtil.nextInt(random(), 1, 100);
     }
     final Set<Integer> valuesSet = new HashSet<>();
-    int numValues = TestUtil.nextInt(random(), 1, 100);
     while (valuesSet.size() < numValues) {
       valuesSet.add(randomIntValue(valueMin, valueMax));
     }
@@ -1285,7 +1341,7 @@ public class TestPointQueries extends LuceneTestCase {
 
               int numExtraValuesToQuery = random().nextInt(20);
               while (valuesToQuery.size() < numValidValuesToQuery + numExtraValuesToQuery) {
-                valuesToQuery.add(randomIntValue(valueMin, valueMax));
+                valuesToQuery.add(random().nextInt());
               }
 
               int expectedCount = 0;


[05/18] lucene-solr git commit: remove debug print

Posted by mi...@apache.org.
remove debug print


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

Branch: refs/heads/master
Commit: 8d88bb7a1313862ae49429cfd38819e90ba09e9b
Parents: 8a4c197
Author: Mike McCandless <mi...@apache.org>
Authored: Tue Feb 23 17:19:11 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Tue Feb 23 17:19:11 2016 -0500

----------------------------------------------------------------------
 .../core/src/java/org/apache/lucene/index/TermsHashPerField.java  | 3 ---
 1 file changed, 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8d88bb7a/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java b/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java
index b06295c..3275a4e 100644
--- a/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java
+++ b/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java
@@ -93,10 +93,7 @@ abstract class TermsHashPerField implements Comparable<TermsHashPerField> {
   /** Collapse the hash table and sort in-place; also sets
    * this.sortedTermIDs to the results */
   public int[] sortPostings() {
-    long t0 = System.nanoTime();
     sortedTermIDs = bytesHash.sort(BytesRef.getUTF8SortedAsUnicodeComparator());
-    // nocommit
-    System.out.println("MKM: field " + fieldInfo.name + " has " + bytesHash.size() + " unique terms " + ((System.nanoTime()-t0)/1000000.0) + " msec to sort");
     return sortedTermIDs;
   }
 


[10/18] lucene-solr git commit: add InetAddressPoint.newSetQuery

Posted by mi...@apache.org.
add InetAddressPoint.newSetQuery


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

Branch: refs/heads/master
Commit: 1fb9de8da5fdf5035d59906b991585716487b225
Parents: dfbcbf9
Author: Mike McCandless <mi...@apache.org>
Authored: Wed Feb 24 15:38:22 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Wed Feb 24 15:38:22 2016 -0500

----------------------------------------------------------------------
 .../lucene/document/InetAddressPoint.java       | 44 ++++++++++++++++++--
 .../lucene/document/TestInetAddressPoint.java   |  2 +
 2 files changed, 43 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1fb9de8d/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressPoint.java b/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressPoint.java
index 1c73d21..d9e9d37 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressPoint.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressPoint.java
@@ -34,6 +34,7 @@ import org.apache.lucene.util.BytesRef;
  *   <li>{@link #newExactQuery newExactQuery()} for matching an exact network address.
  *   <li>{@link #newPrefixQuery newPrefixQuery()} for matching a network based on CIDR prefix.
  *   <li>{@link #newRangeQuery newRangeQuery()} for matching arbitrary network address ranges.
+ *   <li>{@link #newSetQuery newSetQuery()} for matching a set of addresses.
  * </ul>
  * <p>
  * This field supports both IPv4 and IPv6 addresses: IPv4 addresses are converted
@@ -193,11 +194,11 @@ public class InetAddressPoint extends Field {
   public static Query newRangeQuery(String field, InetAddress lowerValue, boolean lowerInclusive, InetAddress upperValue, boolean upperInclusive) {
     byte[][] lowerBytes = new byte[1][];
     if (lowerValue != null) {
-      lowerBytes[0] = InetAddressPoint.encode(lowerValue);
+      lowerBytes[0] = encode(lowerValue);
     }
     byte[][] upperBytes = new byte[1][];
     if (upperValue != null) {
-      upperBytes[0] = InetAddressPoint.encode(upperValue);
+      upperBytes[0] = encode(upperValue);
     }
     return new PointRangeQuery(field, lowerBytes, new boolean[] { lowerInclusive }, upperBytes, new boolean[] { upperInclusive }) {
       @Override
@@ -207,5 +208,42 @@ public class InetAddressPoint extends Field {
     };
   }
 
-  // nocommit newSetQuery
+  /**
+   * Create a query matching any of the specified 1D values.  This is the points equivalent of {@code TermsQuery}.
+   * 
+   * @param field field name. must not be {@code null}.
+   * @param valuesIn all int values to match
+   */
+  public static Query newSetQuery(String field, InetAddress... valuesIn) throws IOException {
+
+    // Don't unexpectedly change the user's incoming values array:
+    InetAddress[] values = valuesIn.clone();
+
+    Arrays.sort(values);
+
+    final BytesRef value = new BytesRef(new byte[BYTES]);
+
+    return new PointInSetQuery(field, 1, BYTES,
+                               new BytesRefIterator() {
+
+                                 int upto;
+
+                                 @Override
+                                 public BytesRef next() {
+                                   if (upto == values.length) {
+                                     return null;
+                                   } else {
+                                     encode(values[upto], value.bytes, 0);
+                                     upto++;
+                                     return value;
+                                   }
+                                 }
+                               }) {
+      @Override
+      protected String toString(byte[] value) {
+        assert value.length == BYTES;
+        return decode(value).getHostAddress();
+      }
+    };
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1fb9de8d/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
index 19b02b2..89a3f84 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
@@ -44,6 +44,8 @@ public class TestInetAddressPoint extends LuceneTestCase {
     assertEquals(1, searcher.count(InetAddressPoint.newExactQuery("field", address)));
     assertEquals(1, searcher.count(InetAddressPoint.newPrefixQuery("field", address, 24)));
     assertEquals(1, searcher.count(InetAddressPoint.newRangeQuery("field", InetAddress.getByName("1.2.3.3"), false, InetAddress.getByName("1.2.3.5"), false)));
+    assertEquals(1, searcher.count(InetAddressPoint.newSetQuery("field", InetAddress.getByName("1.2.3.3")));
+    assertEquals(0, searcher.count(InetAddressPoint.newSetQuery("field", InetAddress.getByName("1.2.3.4")));
 
     reader.close();
     writer.close();


[18/18] lucene-solr git commit: LUCENE-7048: add changes entry

Posted by mi...@apache.org.
LUCENE-7048: add changes entry


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

Branch: refs/heads/master
Commit: b9d46e4fd6de73524be7bb4be0b35c066fed4fae
Parents: 446ce86
Author: Mike McCandless <mi...@apache.org>
Authored: Thu Feb 25 13:11:14 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Thu Feb 25 13:11:14 2016 -0500

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


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b9d46e4f/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 9794611..88f92d1 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -67,6 +67,11 @@ New Features
 * LUCENE-7040: Upgrade morfologik-stemming to version 2.1.0.
   (Dawid Weiss)
 
+* LUCENE-7048: Add XXXPoint.newSetQuery, to create a query that
+  efficiently matches all documents containing any of the specified
+  point values.  This is the analog of TermsQuery, but for points
+  instead.   (Adrien Grand, Robert Muir, Mike McCandless)
+
 API Changes
 
 * LUCENE-6067: Accountable.getChildResources has a default


[09/18] lucene-solr git commit: Merge branch 'master' into point_set_query

Posted by mi...@apache.org.
Merge branch 'master' into point_set_query

Conflicts:
	lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java


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

Branch: refs/heads/master
Commit: dfbcbf96cd3b6142fcc829944015f8de0014f575
Parents: 30fcafd 77e6227
Author: Mike McCandless <mi...@apache.org>
Authored: Wed Feb 24 08:48:48 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Wed Feb 24 08:48:48 2016 -0500

----------------------------------------------------------------------
 .../org/apache/lucene/document/BinaryPoint.java |   9 +-
 .../org/apache/lucene/document/DoublePoint.java |   9 +-
 .../org/apache/lucene/document/FloatPoint.java  |   9 +-
 .../org/apache/lucene/document/IntPoint.java    |   9 +-
 .../org/apache/lucene/document/LongPoint.java   |   9 +-
 .../apache/lucene/search/PointRangeQuery.java   |   7 +-
 .../apache/lucene/search/TestPointQueries.java  |   5 +-
 .../apache/lucene/document/BigIntegerPoint.java |   9 +-
 .../lucene/document/InetAddressPoint.java       |   9 +-
 .../org/apache/lucene/document/LatLonPoint.java | 138 ++++++++++++-
 .../lucene/search/PointInPolygonQuery.java      |   2 +-
 .../apache/lucene/search/PointInRectQuery.java  | 200 -------------------
 .../apache/lucene/document/TestLatLonPoint.java |  56 ++++++
 .../lucene/search/TestLatLonPointQueries.java   |   9 +-
 .../SolrStopwordsCarrot2LexicalDataFactory.java |   6 +-
 .../carrot2/EchoClusteringAlgorithm.java        |   6 +-
 .../carrot2/EchoStemsClusteringAlgorithm.java   |   8 +-
 .../carrot2/EchoTokensClusteringAlgorithm.java  |   6 +-
 ...exicalResourcesCheckClusteringAlgorithm.java |   6 +-
 .../carrot2/MockClusteringAlgorithm.java        |  12 +-
 .../org/apache/solr/handler/BlobHandler.java    |  38 ----
 .../apache/solr/handler/SolrConfigHandler.java  |   3 +-
 .../solr/handler/admin/CollectionsHandler.java  |  20 +-
 .../src/resources/SystemCollectionSchema.xml    |  17 ++
 .../resources/SystemCollectionSolrConfig.xml    |  20 ++
 .../apache/solr/core/TestSolrConfigHandler.java |   4 +-
 26 files changed, 315 insertions(+), 311 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dfbcbf96/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
----------------------------------------------------------------------
diff --cc lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
index 53973c9,2db2d85..e69d0b9
--- a/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
@@@ -16,13 -16,9 +16,14 @@@
   */
  package org.apache.lucene.document;
  
 +import java.io.IOException;
 +import java.util.Arrays;
 +
 +import org.apache.lucene.search.PointInSetQuery;
  import org.apache.lucene.search.PointRangeQuery;
+ import org.apache.lucene.search.Query;
  import org.apache.lucene.util.BytesRef;
 +import org.apache.lucene.util.BytesRefIterator;
  import org.apache.lucene.util.NumericUtils;
  
  /** 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dfbcbf96/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
----------------------------------------------------------------------
diff --cc lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
index d262c18,027dc66..1f39a9f
--- a/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
@@@ -16,13 -16,9 +16,14 @@@
   */
  package org.apache.lucene.document;
  
 +import java.io.IOException;
 +import java.util.Arrays;
 +
 +import org.apache.lucene.search.PointInSetQuery;
  import org.apache.lucene.search.PointRangeQuery;
+ import org.apache.lucene.search.Query;
  import org.apache.lucene.util.BytesRef;
 +import org.apache.lucene.util.BytesRefIterator;
  import org.apache.lucene.util.NumericUtils;
  
  /** 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dfbcbf96/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
----------------------------------------------------------------------
diff --cc lucene/core/src/java/org/apache/lucene/document/IntPoint.java
index cdd10c7,2d411e9..a78c30d
--- a/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
@@@ -16,13 -16,9 +16,14 @@@
   */
  package org.apache.lucene.document;
  
 +import java.io.IOException;
 +import java.util.Arrays;
 +
 +import org.apache.lucene.search.PointInSetQuery;
  import org.apache.lucene.search.PointRangeQuery;
+ import org.apache.lucene.search.Query;
  import org.apache.lucene.util.BytesRef;
 +import org.apache.lucene.util.BytesRefIterator;
  import org.apache.lucene.util.NumericUtils;
  
  /** 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dfbcbf96/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
----------------------------------------------------------------------
diff --cc lucene/core/src/java/org/apache/lucene/document/LongPoint.java
index 9c0ba3f,69b511d..d0c5655
--- a/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
@@@ -16,13 -16,9 +16,14 @@@
   */
  package org.apache.lucene.document;
  
 +import java.io.IOException;
 +import java.util.Arrays;
 +
 +import org.apache.lucene.search.PointInSetQuery;
  import org.apache.lucene.search.PointRangeQuery;
+ import org.apache.lucene.search.Query;
  import org.apache.lucene.util.BytesRef;
 +import org.apache.lucene.util.BytesRefIterator;
  import org.apache.lucene.util.NumericUtils;
  
  /** 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dfbcbf96/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dfbcbf96/lucene/sandbox/src/java/org/apache/lucene/document/BigIntegerPoint.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dfbcbf96/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressPoint.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dfbcbf96/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java
----------------------------------------------------------------------
diff --cc lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java
index 9e18c19,0ea9efb..dca5f97
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java
@@@ -83,6 -128,89 +128,91 @@@ public class LatLonPoint extends Field 
    public static double decodeLon(int x) {
      return x / LON_SCALE;
    }
 +
 +  // nocommit newSetQuery
+   
+   /** Turns quantized value from byte array back into a double. */
+   public static double decodeLon(byte[] src, int offset) {
+     return decodeLon(NumericUtils.bytesToInt(src, offset));
+   }
+   
+   /** sugar encodes a single point as a 2D byte array */
+   private static byte[][] encode(double lat, double lon) {
+     byte[][] bytes = new byte[2][];
+     bytes[0] = new byte[4];
+     NumericUtils.intToBytes(encodeLat(lat), bytes[0], 0);
+     bytes[1] = new byte[4];
+     NumericUtils.intToBytes(encodeLon(lon), bytes[1], 0);
+     return bytes;
+   }
+    
+   // static methods for generating queries
+ 
+   /**
+    * Create a query for matching a bounding box.
+    * <p>
+    * The box may cross over the dateline.
+    */
+   public static Query newBoxQuery(String field, double minLat, double maxLat, double minLon, double maxLon) {
+     if (GeoUtils.isValidLat(minLat) == false) {
+       throw new IllegalArgumentException("minLat=" + minLat + " is not a valid latitude");
+     }
+     if (GeoUtils.isValidLat(maxLat) == false) {
+       throw new IllegalArgumentException("maxLat=" + maxLat + " is not a valid latitude");
+     }
+     if (GeoUtils.isValidLon(minLon) == false) {
+       throw new IllegalArgumentException("minLon=" + minLon + " is not a valid longitude");
+     }
+     if (GeoUtils.isValidLon(maxLon) == false) {
+       throw new IllegalArgumentException("maxLon=" + maxLon + " is not a valid longitude");
+     }
+     
+     byte[][] lower = encode(minLat, minLon);
+     byte[][] upper = encode(maxLat, maxLon);
+     // Crosses date line: we just rewrite into OR of two bboxes, with longitude as an open range:
+     if (maxLon < minLon) {
+       // Disable coord here because a multi-valued doc could match both rects and get unfairly boosted:
+       BooleanQuery.Builder q = new BooleanQuery.Builder();
+       q.setDisableCoord(true);
+ 
+       // E.g.: maxLon = -179, minLon = 179
+       byte[][] leftOpen = new byte[2][];
+       leftOpen[0] = lower[0];
+       // leave longitude open (null)
+       Query left = newBoxInternal(field, leftOpen, upper);
+       q.add(new BooleanClause(left, BooleanClause.Occur.SHOULD));
+       byte[][] rightOpen = new byte[2][];
+       rightOpen[0] = upper[0];
+       // leave longitude open (null)
+       Query right = newBoxInternal(field, lower, rightOpen);
+       q.add(new BooleanClause(right, BooleanClause.Occur.SHOULD));
+       return new ConstantScoreQuery(q.build());
+     } else {
+       return newBoxInternal(field, lower, upper);
+     }
+   }
+   
+   private static Query newBoxInternal(String field, byte[][] min, byte[][] max) {
+     return new PointRangeQuery(field, min, new boolean[] { true, true }, max, new boolean[] { false, false }) {
+       @Override
+       protected String toString(int dimension, byte[] value) {
+         if (dimension == 0) {
+           return Double.toString(decodeLat(value, 0));
+         } else if (dimension == 1) {
+           return Double.toString(decodeLon(value, 0));
+         } else {
+           throw new AssertionError();
+         }
+       }
+     };
+   }
+   
+   /** 
+    * Create a query for matching a polygon.
+    * <p>
+    * The supplied {@code polyLats}/{@code polyLons} must be clockwise or counter-clockwise.
+    */
+   public static Query newPolygonQuery(String field, double[] polyLats, double[] polyLons) {
+     return new PointInPolygonQuery(field, polyLats, polyLons);
+   }
  }


[15/18] lucene-solr git commit: get multi-dim point-in-set testcase working; add explicit tests for 'no points queried' cases; fix last nocommits

Posted by mi...@apache.org.
get multi-dim point-in-set testcase working; add explicit tests for 'no points queried' cases; fix last nocommits


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

Branch: refs/heads/master
Commit: a1f5269280258585dc2c58c8b76999287368599c
Parents: 41a336f
Author: Mike McCandless <mi...@apache.org>
Authored: Thu Feb 25 05:21:08 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Thu Feb 25 05:21:08 2016 -0500

----------------------------------------------------------------------
 .../org/apache/lucene/document/BinaryPoint.java |  12 +-
 .../apache/lucene/search/PointInSetQuery.java   |  10 +-
 .../apache/lucene/search/TestPointQueries.java  | 177 ++++++++++++++++++-
 .../lucene/document/TestBigIntegerPoint.java    |   1 +
 .../lucene/document/TestInetAddressPoint.java   |   1 +
 5 files changed, 185 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1f52692/lucene/core/src/java/org/apache/lucene/document/BinaryPoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/BinaryPoint.java b/lucene/core/src/java/org/apache/lucene/document/BinaryPoint.java
index b8cbc1e..9c4de99 100644
--- a/lucene/core/src/java/org/apache/lucene/document/BinaryPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/BinaryPoint.java
@@ -20,6 +20,7 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.Comparator;
 
+import org.apache.lucene.search.MatchNoDocsQuery;
 import org.apache.lucene.search.PointInSetQuery;
 import org.apache.lucene.search.PointRangeQuery;
 import org.apache.lucene.search.Query;
@@ -224,6 +225,11 @@ public final class BinaryPoint extends Field {
       }
     }
 
+    if (bytesPerDim == -1) {
+      // There are no points, and we cannot guess the bytesPerDim here, so we return an equivalent query:
+      return new MatchNoDocsQuery();
+    }
+
     // Don't unexpectedly change the user's incoming values array:
     byte[][] values = valuesIn.clone();
 
@@ -235,12 +241,6 @@ public final class BinaryPoint extends Field {
                   }
                 });
 
-    // Silliness:
-    if (bytesPerDim == -1) {
-      // nocommit make sure this is tested
-      bytesPerDim = 1;
-    }
-
     final BytesRef value = new BytesRef(new byte[bytesPerDim]);
     
     return new PointInSetQuery(field, 1, bytesPerDim,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1f52692/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java b/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
index d34e3f7..348f598 100644
--- a/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
@@ -50,7 +50,7 @@ public class PointInSetQuery extends Query {
   final int numDims;
   final int bytesPerDim;
 
-  /** In the 1D case, the {@code packedPoints} iterator must be in sorted order. */
+  /** The {@code packedPoints} iterator must be in sorted order. */
   protected PointInSetQuery(String field, int numDims, int bytesPerDim, BytesRefIterator packedPoints) throws IOException {
     this.field = field;
     if (bytesPerDim < 1 || bytesPerDim > PointValues.MAX_NUM_BYTES) {
@@ -78,8 +78,8 @@ public class PointInSetQuery extends Query {
         int cmp = previous.get().compareTo(current);
         if (cmp == 0) {
           continue; // deduplicate
-        } else if (numDims == 1 && cmp > 0) {
-          throw new IllegalArgumentException("numDims=1 and values are out of order: saw " + previous + " before " + current);
+        } else if (cmp > 0) {
+          throw new IllegalArgumentException("values are out of order: saw " + previous + " before " + current);
         }
       }
       builder.add(field, current);
@@ -230,7 +230,6 @@ public class PointInSetQuery extends Query {
 
     private final DocIdSetBuilder result;
     private final int[] hitCount;
-    public BytesRef point;
     private final byte[] pointBytes;
 
     public SinglePointVisitor(int[] hitCount, DocIdSetBuilder result) {
@@ -258,7 +257,7 @@ public class PointInSetQuery extends Query {
 
     @Override
     public void visit(int docID, byte[] packedValue) {
-      assert packedValue.length == point.length;
+      assert packedValue.length == pointBytes.length;
       if (Arrays.equals(packedValue, pointBytes)) {
         // The point for this doc matches the point we are querying on
         hitCount[0]++;
@@ -292,7 +291,6 @@ public class PointInSetQuery extends Query {
       if (crosses) {
         return Relation.CELL_CROSSES_QUERY;
       } else {
-        // nocommit make sure tests hit this case:
         // NOTE: we only hit this if we are on a cell whose min and max values are exactly equal to our point,
         // which can easily happen if many docs share this one value
         return Relation.CELL_INSIDE_QUERY;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1f52692/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 095ddc9..4a232a6 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
@@ -22,6 +22,7 @@ import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.BitSet;
+import java.util.Comparator;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -64,6 +65,7 @@ import org.apache.lucene.util.BytesRefIterator;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.NumericUtils;
+import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.TestUtil;
 import org.junit.BeforeClass;
 
@@ -1281,8 +1283,6 @@ public class TestPointQueries extends LuceneTestCase {
     iwc.setCodec(getCodec());
     RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
 
-    // nocommit multi-valued too
-
     int[] docValues = new int[numDocs];
     for(int i=0;i<numDocs;i++) {
       int x = values[random().nextInt(values.length)];
@@ -1370,7 +1370,150 @@ public class TestPointQueries extends LuceneTestCase {
     IOUtils.close(r, dir);
   }
 
-  // nocommit need 2D test too
+  // TODO: in the future, if there is demand for real usage, we can "graduate" this test-only query factory as IntPoint.newMultiSetQuery or
+  // something (and same for other XXXPoint classes):
+  private static Query newMultiDimIntSetQuery(String field, final int numDims, int... valuesIn) throws IOException {
+    if (valuesIn.length % numDims != 0) {
+      throw new IllegalArgumentException("incongruent number of values: valuesIn.length=" + valuesIn.length + " but numDims=" + numDims);
+    }
+
+    // Pack all values:
+    byte[][] packedValues = new byte[valuesIn.length / numDims][];
+    for(int i=0;i<packedValues.length;i++) {
+      byte[] packedValue = new byte[numDims * Integer.BYTES];
+      packedValues[i] = packedValue;
+      for(int dim=0;dim<numDims;dim++) {
+        IntPoint.encodeDimension(valuesIn[i*numDims+dim], packedValue, dim*Integer.BYTES);
+      }
+    }
+
+    // Sort:
+    Arrays.sort(packedValues,
+                new Comparator<byte[]>() {
+                  @Override
+                  public int compare(byte[] a, byte[] b) {
+                    return StringHelper.compare(a.length, a, 0, b, 0);
+                  }
+                });
+
+    final BytesRef value = new BytesRef();
+    value.length = numDims * Integer.BYTES;
+
+    return new PointInSetQuery(field,
+                               numDims,
+                               Integer.BYTES,
+                               new BytesRefIterator() {
+                                 int upto;
+                                 @Override
+                                 public BytesRef next() {
+                                   if (upto >= packedValues.length) {
+                                     return null;
+                                   }
+                                   value.bytes = packedValues[upto];
+                                   upto++;
+                                   return value;
+                                 }
+                               }) {
+      @Override
+      protected String toString(byte[] value) {
+        assert value.length == numDims * Integer.BYTES;
+        StringBuilder sb = new StringBuilder();
+        for(int dim=0;dim<numDims;dim++) {
+          if (dim > 0) {
+            sb.append(',');
+          }
+          sb.append(Integer.toString(IntPoint.decodeDimension(value, dim*Integer.BYTES)));
+        }
+
+        return sb.toString();
+      }
+    };
+  }
+
+  public void testBasicMultiDimPointInSetQuery() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = newIndexWriterConfig();
+    iwc.setCodec(getCodec());
+    IndexWriter w = new IndexWriter(dir, iwc);
+
+    Document doc = new Document();
+    doc.add(new IntPoint("int", 17, 42));
+    w.addDocument(doc);
+    IndexReader r = DirectoryReader.open(w);
+    IndexSearcher s = newSearcher(r);
+
+    assertEquals(0, s.count(newMultiDimIntSetQuery("int", 2, 17, 41)));
+    assertEquals(1, s.count(newMultiDimIntSetQuery("int", 2, 17, 42)));
+    assertEquals(1, s.count(newMultiDimIntSetQuery("int", 2, -7, -7, 17, 42)));
+    assertEquals(1, s.count(newMultiDimIntSetQuery("int", 2, 17, 42, -14, -14)));
+
+    w.close();
+    r.close();
+    dir.close();
+  }
+
+  public void testBasicMultiValueMultiDimPointInSetQuery() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = newIndexWriterConfig();
+    iwc.setCodec(getCodec());
+    IndexWriter w = new IndexWriter(dir, iwc);
+
+    Document doc = new Document();
+    doc.add(new IntPoint("int", 17, 42));
+    doc.add(new IntPoint("int", 34, 79));
+    w.addDocument(doc);
+    IndexReader r = DirectoryReader.open(w);
+    IndexSearcher s = newSearcher(r);
+
+    assertEquals(0, s.count(newMultiDimIntSetQuery("int", 2, 17, 41)));
+    assertEquals(1, s.count(newMultiDimIntSetQuery("int", 2, 17, 42)));
+    assertEquals(1, s.count(newMultiDimIntSetQuery("int", 2, 17, 42, 34, 79)));
+    assertEquals(1, s.count(newMultiDimIntSetQuery("int", 2, -7, -7, 17, 42)));
+    assertEquals(1, s.count(newMultiDimIntSetQuery("int", 2, -7, -7, 34, 79)));
+    assertEquals(1, s.count(newMultiDimIntSetQuery("int", 2, 17, 42, -14, -14)));
+
+    assertEquals("int:{-14,-14 17,42}", newMultiDimIntSetQuery("int", 2, 17, 42, -14, -14).toString());
+
+    w.close();
+    r.close();
+    dir.close();
+  }
+
+  public void testManyEqualValuesMultiDimPointInSetQuery() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = newIndexWriterConfig();
+    iwc.setCodec(getCodec());
+    IndexWriter w = new IndexWriter(dir, iwc);
+
+    int zeroCount = 0;
+    for(int i=0;i<10000;i++) {
+      int x = random().nextInt(2);
+      if (x == 0) {
+        zeroCount++;
+      }
+      Document doc = new Document();
+      doc.add(new IntPoint("int", x, x));
+      w.addDocument(doc);
+    }
+    IndexReader r = DirectoryReader.open(w);
+    IndexSearcher s = newSearcher(r);
+
+    assertEquals(zeroCount, s.count(newMultiDimIntSetQuery("int", 2, 0, 0)));
+    assertEquals(10000-zeroCount, s.count(newMultiDimIntSetQuery("int", 2, 1, 1)));
+    assertEquals(0, s.count(newMultiDimIntSetQuery("int", 2, 2, 2)));
+
+    w.close();
+    r.close();
+    dir.close();
+  }
+
+  public void testInvalidMultiDimPointInSetQuery() throws Exception {
+    IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+                                                     () -> {
+                                                       newMultiDimIntSetQuery("int", 2, 3, 4, 5);
+                                                     });
+    assertEquals("incongruent number of values: valuesIn.length=3 but numDims=2", expected.getMessage());
+  }
 
   public void testBasicPointInSetQuery() throws Exception {
     Directory dir = newDirectory();
@@ -1432,7 +1575,6 @@ public class TestPointQueries extends LuceneTestCase {
     assertEquals(3, s.count(DoublePoint.newSetQuery("double", 17, 20, 42, 97)));
     assertEquals(3, s.count(DoublePoint.newSetQuery("double", 17, 105, 42, 97)));
 
-    // nocommit make sure invalid bytes length hits iae
     assertEquals(0, s.count(BinaryPoint.newSetQuery("bytes", new byte[] {0, 16})));
     assertEquals(1, s.count(BinaryPoint.newSetQuery("bytes", new byte[] {0, 17})));
     assertEquals(3, s.count(BinaryPoint.newSetQuery("bytes", new byte[] {0, 17}, new byte[] {0, 97}, new byte[] {0, 42})));
@@ -1501,6 +1643,33 @@ public class TestPointQueries extends LuceneTestCase {
     dir.close();
   }
 
+  public void testEmptyPointInSetQuery() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = newIndexWriterConfig();
+    iwc.setCodec(getCodec());
+    IndexWriter w = new IndexWriter(dir, iwc);
+
+    Document doc = new Document();
+    doc.add(new IntPoint("int", 17));
+    doc.add(new LongPoint("long", 17L));
+    doc.add(new FloatPoint("float", 17.0f));
+    doc.add(new DoublePoint("double", 17.0));
+    doc.add(new BinaryPoint("bytes", new byte[] {0, 17}));
+    w.addDocument(doc);
+
+    IndexReader r = DirectoryReader.open(w);
+    IndexSearcher s = newSearcher(r);
+    assertEquals(0, s.count(IntPoint.newSetQuery("int")));
+    assertEquals(0, s.count(LongPoint.newSetQuery("long")));
+    assertEquals(0, s.count(FloatPoint.newSetQuery("float")));
+    assertEquals(0, s.count(DoublePoint.newSetQuery("double")));
+    assertEquals(0, s.count(BinaryPoint.newSetQuery("bytes")));
+
+    w.close();
+    r.close();
+    dir.close();
+  }
+
   public void testPointInSetQueryManyEqualValues() throws Exception {
     Directory dir = newDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1f52692/lucene/sandbox/src/test/org/apache/lucene/document/TestBigIntegerPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestBigIntegerPoint.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestBigIntegerPoint.java
index 6b0d696..b5d8022 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestBigIntegerPoint.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestBigIntegerPoint.java
@@ -45,6 +45,7 @@ public class TestBigIntegerPoint extends LuceneTestCase {
     assertEquals(1, searcher.count(BigIntegerPoint.newRangeQuery("field", large.subtract(BigInteger.ONE), false, large.add(BigInteger.ONE), false)));
     assertEquals(1, searcher.count(BigIntegerPoint.newSetQuery("field", large)));
     assertEquals(0, searcher.count(BigIntegerPoint.newSetQuery("field", large.subtract(BigInteger.ONE))));
+    assertEquals(0, searcher.count(BigIntegerPoint.newSetQuery("field"));
 
     reader.close();
     writer.close();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1f52692/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
index 500edef..b2f41c4 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
@@ -46,6 +46,7 @@ public class TestInetAddressPoint extends LuceneTestCase {
     assertEquals(1, searcher.count(InetAddressPoint.newRangeQuery("field", InetAddress.getByName("1.2.3.3"), false, InetAddress.getByName("1.2.3.5"), false)));
     assertEquals(1, searcher.count(InetAddressPoint.newSetQuery("field", InetAddress.getByName("1.2.3.4"))));
     assertEquals(0, searcher.count(InetAddressPoint.newSetQuery("field", InetAddress.getByName("1.2.3.3"))));
+    assertEquals(0, searcher.count(InetAddressPoint.newSetQuery("field"));
 
     reader.close();
     writer.close();


[03/18] lucene-solr git commit: more tests; move factory method to IntPoint

Posted by mi...@apache.org.
more tests; move factory method to IntPoint


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

Branch: refs/heads/master
Commit: 1654818e9814b99a75d2d4f4ac590813fab2f10a
Parents: 96ed42b
Author: Mike McCandless <mi...@apache.org>
Authored: Tue Feb 23 17:12:50 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Tue Feb 23 17:12:50 2016 -0500

----------------------------------------------------------------------
 .../org/apache/lucene/document/IntPoint.java    | 40 +++++++++++
 .../apache/lucene/search/PointInSetQuery.java   | 66 ++++++++----------
 .../apache/lucene/search/TestPointQueries.java  | 70 ++++++++++++++++++--
 3 files changed, 133 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1654818e/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/IntPoint.java b/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
index be91bfd..42091d9 100644
--- a/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
@@ -16,7 +16,12 @@
  */
 package org.apache.lucene.document;
 
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.lucene.search.PointInSetQuery;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefIterator;
 import org.apache.lucene.util.NumericUtils;
 
 /** An int field that is indexed dimensionally such that finding
@@ -88,6 +93,41 @@ public final class IntPoint extends Field {
     super(name, pack(point), getType(point.length));
   }
   
+  /** Returns a query efficiently finding all documents that indexed the provided 1D int values */
+  public static PointInSetQuery newSetQuery(String field, int... valuesIn) throws IOException {
+
+    // Don't unexpectedly change the user's incoming array:
+    int[] values = valuesIn.clone();
+
+    Arrays.sort(values);
+
+    final BytesRef value = new BytesRef(new byte[Integer.BYTES]);
+    value.length = Integer.BYTES;
+
+    return new PointInSetQuery(field, 1, Integer.BYTES,
+                               new BytesRefIterator() {
+
+                                 int upto;
+
+                                 @Override
+                                 public BytesRef next() {
+                                   if (upto == values.length) {
+                                     return null;
+                                   } else {
+                                     IntPoint.encodeDimension(values[upto], value.bytes, 0);
+                                     upto++;
+                                     return value;
+                                   }
+                                 }
+                               }) {
+      @Override
+      protected String toString(byte[] value) {
+        assert value.length == Integer.BYTES;
+        return Integer.toString(decodeDimension(value, 0));
+      }
+    };
+  }
+
   @Override
   public String toString() {
     StringBuilder result = new StringBuilder();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1654818e/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java b/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
index bbc9a54..adb9c53 100644
--- a/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
@@ -80,40 +80,6 @@ public class PointInSetQuery extends Query {
     sortedPackedPointsHashCode = sortedPackedPoints.hashCode();
   }
 
-  /** Use in the 1D case when you indexed 1D int values using {@link org.apache.lucene.document.IntPoint} */
-  public static PointInSetQuery newIntSet(String field, int... valuesIn) {
-
-    // Don't unexpectedly change the user's incoming array:
-    int[] values = valuesIn.clone();
-
-    Arrays.sort(values);
-
-    final BytesRef value = new BytesRef(new byte[Integer.BYTES]);
-    value.length = Integer.BYTES;
-
-    try {
-      return new PointInSetQuery(field, 1, Integer.BYTES,
-                                 new BytesRefIterator() {
-
-                                   int upto;
-
-                                   @Override
-                                   public BytesRef next() {
-                                     if (upto == values.length) {
-                                       return null;
-                                     } else {
-                                       IntPoint.encodeDimension(values[upto], value.bytes, 0);
-                                       upto++;
-                                       return value;
-                                     }
-                                   }
-                                 });
-    } catch (IOException bogus) {
-      // Should never happen ;)
-      throw new RuntimeException(bogus);
-    }
-  }
-
   @Override
   public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
 
@@ -354,18 +320,44 @@ public class PointInSetQuery extends Query {
     sb.append(getClass().getSimpleName());
     sb.append(':');
     if (this.field.equals(field) == false) {
-      sb.append("field=");
+      sb.append(" field=");
       sb.append(this.field);
       sb.append(':');
     }
 
+    sb.append(" points:");
+
     TermIterator iterator = sortedPackedPoints.iterator();
+    byte[] pointBytes = new byte[numDims * bytesPerDim];
     for (BytesRef point = iterator.next(); point != null; point = iterator.next()) {
       sb.append(' ');
-      // nocommit fix me to convert back to the numbers/etc.:
-      sb.append(point);
+      System.arraycopy(point.bytes, point.offset, pointBytes, 0, pointBytes.length);
+      sb.append(toString(pointBytes));
     }
 
     return sb.toString();
   }
+
+  /**
+   * Returns a string of a single value in a human-readable format for debugging.
+   * This is used by {@link #toString()}.
+   *
+   * The default implementation encodes the individual byte values.
+   *
+   * @param value single value, never null
+   * @return human readable value for debugging
+   */
+  protected String toString(byte[] value) {
+    assert value != null;
+    StringBuilder sb = new StringBuilder();
+    sb.append("binary(");
+    for (int i = 0; i < value.length; i++) {
+      if (i > 0) {
+        sb.append(' ');
+      }
+      sb.append(Integer.toHexString(value[i] & 0xFF));
+    }
+    sb.append(')');
+    return sb.toString();
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1654818e/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 ea432f7..c48cacb 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
@@ -1139,12 +1139,70 @@ public class TestPointQueries extends LuceneTestCase {
 
     IndexReader r = DirectoryReader.open(w);
     IndexSearcher s = newSearcher(r);
-    assertEquals(0, s.count(PointInSetQuery.newIntSet("int", 16)));
-    assertEquals(1, s.count(PointInSetQuery.newIntSet("int", 17)));
-    assertEquals(3, s.count(PointInSetQuery.newIntSet("int", 17, 97, 42)));
-    assertEquals(3, s.count(PointInSetQuery.newIntSet("int", -7, 17, 42, 97)));
-    assertEquals(3, s.count(PointInSetQuery.newIntSet("int", 17, 20, 42, 97)));
-    assertEquals(3, s.count(PointInSetQuery.newIntSet("int", 17, 105, 42, 97)));
+    assertEquals(0, s.count(IntPoint.newSetQuery("int", 16)));
+    assertEquals(1, s.count(IntPoint.newSetQuery("int", 17)));
+    assertEquals(3, s.count(IntPoint.newSetQuery("int", 17, 97, 42)));
+    assertEquals(3, s.count(IntPoint.newSetQuery("int", -7, 17, 42, 97)));
+    assertEquals(3, s.count(IntPoint.newSetQuery("int", 17, 20, 42, 97)));
+    assertEquals(3, s.count(IntPoint.newSetQuery("int", 17, 105, 42, 97)));
+    w.close();
+    r.close();
+    dir.close();
+  }
+
+  public void testPointInSetQueryManyEqualValues() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = newIndexWriterConfig();
+    iwc.setCodec(getCodec());
+    IndexWriter w = new IndexWriter(dir, iwc);
+
+    int zeroCount = 0;
+    for(int i=0;i<10000;i++) {
+      int x = random().nextInt(2);
+      if (x == 0) {
+        zeroCount++;
+      }
+      Document doc = new Document();
+      doc.add(new IntPoint("int", x));
+      w.addDocument(doc);
+    }
+
+    IndexReader r = DirectoryReader.open(w);
+    IndexSearcher s = newSearcher(r);
+    assertEquals(zeroCount, s.count(IntPoint.newSetQuery("int", 0)));
+    assertEquals(zeroCount, s.count(IntPoint.newSetQuery("int", 0, -7)));
+    assertEquals(zeroCount, s.count(IntPoint.newSetQuery("int", 7, 0)));
+    assertEquals(10000-zeroCount, s.count(IntPoint.newSetQuery("int", 1)));
+    assertEquals(0, s.count(IntPoint.newSetQuery("int", 2)));
+    w.close();
+    r.close();
+    dir.close();
+  }
+
+  public void testPointInSetQueryManyEqualValuesBigGap() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = newIndexWriterConfig();
+    iwc.setCodec(getCodec());
+    IndexWriter w = new IndexWriter(dir, iwc);
+
+    int zeroCount = 0;
+    for(int i=0;i<10000;i++) {
+      int x = 200 * random().nextInt(2);
+      if (x == 0) {
+        zeroCount++;
+      }
+      Document doc = new Document();
+      doc.add(new IntPoint("int", x));
+      w.addDocument(doc);
+    }
+
+    IndexReader r = DirectoryReader.open(w);
+    IndexSearcher s = newSearcher(r);
+    assertEquals(zeroCount, s.count(IntPoint.newSetQuery("int", 0)));
+    assertEquals(zeroCount, s.count(IntPoint.newSetQuery("int", 0, -7)));
+    assertEquals(zeroCount, s.count(IntPoint.newSetQuery("int", 7, 0)));
+    assertEquals(10000-zeroCount, s.count(IntPoint.newSetQuery("int", 200)));
+    assertEquals(0, s.count(IntPoint.newSetQuery("int", 2)));
     w.close();
     r.close();
     dir.close();


[08/18] lucene-solr git commit: add newSetQuery to DoublePoint and FloatPoint

Posted by mi...@apache.org.
add newSetQuery to DoublePoint and FloatPoint


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

Branch: refs/heads/master
Commit: 30fcafdfd944c97f5c1fdaa9a310f04e7c1b2f2e
Parents: 79c8bce
Author: Mike McCandless <mi...@apache.org>
Authored: Wed Feb 24 08:48:10 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Wed Feb 24 08:48:10 2016 -0500

----------------------------------------------------------------------
 .../org/apache/lucene/document/DoublePoint.java | 44 ++++++++++
 .../org/apache/lucene/document/FloatPoint.java  | 47 +++++++++++
 .../org/apache/lucene/document/IntPoint.java    |  5 +-
 .../org/apache/lucene/document/LongPoint.java   |  5 +-
 .../apache/lucene/search/TestPointQueries.java  | 86 ++++++++++++++++++--
 .../apache/lucene/document/BigIntegerPoint.java |  2 +
 .../lucene/document/InetAddressPoint.java       |  2 +
 .../org/apache/lucene/document/LatLonPoint.java |  2 +
 8 files changed, 180 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/30fcafdf/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java b/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
index ee0d6f2..53973c9 100644
--- a/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
@@ -16,8 +16,13 @@
  */
 package org.apache.lucene.document;
 
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.lucene.search.PointInSetQuery;
 import org.apache.lucene.search.PointRangeQuery;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefIterator;
 import org.apache.lucene.util.NumericUtils;
 
 /** 
@@ -212,4 +217,43 @@ public final class DoublePoint extends Field {
       }
     };
   }
+
+  /**
+   * Create a query matching any of the specified 1D values.  This is the points equivalent of {@code TermsQuery}.
+   * 
+   * @param field field name. must not be {@code null}.
+   * @param valuesIn all int values to match
+   */
+  public static Query newSetQuery(String field, double... valuesIn) throws IOException {
+
+    // Don't unexpectedly change the user's incoming values array:
+    double[] values = valuesIn.clone();
+
+    Arrays.sort(values);
+
+    final BytesRef value = new BytesRef(new byte[Double.BYTES]);
+
+    return new PointInSetQuery(field, 1, Double.BYTES,
+                               new BytesRefIterator() {
+
+                                 int upto;
+
+                                 @Override
+                                 public BytesRef next() {
+                                   if (upto == values.length) {
+                                     return null;
+                                   } else {
+                                     encodeDimension(values[upto], value.bytes, 0);
+                                     upto++;
+                                     return value;
+                                   }
+                                 }
+                               }) {
+      @Override
+      protected String toString(byte[] value) {
+        assert value.length == Double.BYTES;
+        return Double.toString(decodeDimension(value, 0));
+      }
+    };
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/30fcafdf/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java b/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
index 34216b4..d262c18 100644
--- a/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
@@ -16,8 +16,13 @@
  */
 package org.apache.lucene.document;
 
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.lucene.search.PointInSetQuery;
 import org.apache.lucene.search.PointRangeQuery;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefIterator;
 import org.apache.lucene.util.NumericUtils;
 
 /** 
@@ -212,4 +217,46 @@ public final class FloatPoint extends Field {
       }
     };
   }
+
+  /**
+   * Create a query matching any of the specified 1D values.  This is the points equivalent of {@code TermsQuery}.
+   * 
+   * @param field field name. must not be {@code null}.
+   * @param valuesIn all int values to match
+   */
+  public static Query newSetQuery(String field, float... valuesIn) throws IOException {
+
+    // Don't unexpectedly change the user's incoming values array:
+    float[] values = valuesIn.clone();
+
+    Arrays.sort(values);
+
+    System.out.println("VALUES: " + Arrays.toString(values));
+
+    final BytesRef value = new BytesRef(new byte[Float.BYTES]);
+
+    return new PointInSetQuery(field, 1, Float.BYTES,
+                               new BytesRefIterator() {
+
+                                 int upto;
+
+                                 @Override
+                                 public BytesRef next() {
+                                   if (upto == values.length) {
+                                     return null;
+                                   } else {
+                                     encodeDimension(values[upto], value.bytes, 0);
+                                     upto++;
+                                     System.out.println("ret: " + value);
+                                     return value;
+                                   }
+                                 }
+                               }) {
+      @Override
+      protected String toString(byte[] value) {
+        assert value.length == Float.BYTES;
+        return Float.toString(decodeDimension(value, 0));
+      }
+    };
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/30fcafdf/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/IntPoint.java b/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
index 2b255d7..cdd10c7 100644
--- a/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
@@ -224,7 +224,7 @@ public final class IntPoint extends Field {
    * @param field field name. must not be {@code null}.
    * @param valuesIn all int values to match
    */
-  public static PointInSetQuery newSetQuery(String field, int... valuesIn) throws IOException {
+  public static Query newSetQuery(String field, int... valuesIn) throws IOException {
 
     // Don't unexpectedly change the user's incoming values array:
     int[] values = valuesIn.clone();
@@ -232,7 +232,6 @@ public final class IntPoint extends Field {
     Arrays.sort(values);
 
     final BytesRef value = new BytesRef(new byte[Integer.BYTES]);
-    value.length = Integer.BYTES;
 
     return new PointInSetQuery(field, 1, Integer.BYTES,
                                new BytesRefIterator() {
@@ -244,7 +243,7 @@ public final class IntPoint extends Field {
                                    if (upto == values.length) {
                                      return null;
                                    } else {
-                                     IntPoint.encodeDimension(values[upto], value.bytes, 0);
+                                     encodeDimension(values[upto], value.bytes, 0);
                                      upto++;
                                      return value;
                                    }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/30fcafdf/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/LongPoint.java b/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
index 16f7e6c..9c0ba3f 100644
--- a/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
@@ -224,7 +224,7 @@ public final class LongPoint extends Field {
    * @param field field name. must not be {@code null}.
    * @param valuesIn all int values to match
    */
-  public static PointInSetQuery newSetQuery(String field, long... valuesIn) throws IOException {
+  public static Query newSetQuery(String field, long... valuesIn) throws IOException {
 
     // Don't unexpectedly change the user's incoming values array:
     long[] values = valuesIn.clone();
@@ -232,7 +232,6 @@ public final class LongPoint extends Field {
     Arrays.sort(values);
 
     final BytesRef value = new BytesRef(new byte[Long.BYTES]);
-    value.length = Long.BYTES;
 
     return new PointInSetQuery(field, 1, Long.BYTES,
                                new BytesRefIterator() {
@@ -244,7 +243,7 @@ public final class LongPoint extends Field {
                                    if (upto == values.length) {
                                      return null;
                                    } else {
-                                     LongPoint.encodeDimension(values[upto], value.bytes, 0);
+                                     encodeDimension(values[upto], value.bytes, 0);
                                      upto++;
                                      return value;
                                    }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/30fcafdf/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 843a929..e439cc7 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
@@ -1270,17 +1270,23 @@ public class TestPointQueries extends LuceneTestCase {
 
     Document doc = new Document();
     doc.add(new IntPoint("int", 17));
-    doc.add(new LongPoint("long", 17));
+    doc.add(new LongPoint("long", 17L));
+    doc.add(new FloatPoint("float", 17.0f));
+    doc.add(new DoublePoint("double", 17.0));
     w.addDocument(doc);
 
     doc = new Document();
     doc.add(new IntPoint("int", 42));
-    doc.add(new LongPoint("long", 42));
+    doc.add(new LongPoint("long", 42L));
+    doc.add(new FloatPoint("float", 42.0f));
+    doc.add(new DoublePoint("double", 42.0));
     w.addDocument(doc);
 
     doc = new Document();
     doc.add(new IntPoint("int", 97));
-    doc.add(new LongPoint("long", 97));
+    doc.add(new LongPoint("long", 97L));
+    doc.add(new FloatPoint("float", 97.0f));
+    doc.add(new DoublePoint("double", 97.0));
     w.addDocument(doc);
 
     IndexReader r = DirectoryReader.open(w);
@@ -1299,6 +1305,20 @@ public class TestPointQueries extends LuceneTestCase {
     assertEquals(3, s.count(LongPoint.newSetQuery("long", 17, 20, 42, 97)));
     assertEquals(3, s.count(LongPoint.newSetQuery("long", 17, 105, 42, 97)));
 
+    assertEquals(0, s.count(LongPoint.newSetQuery("float", 16)));
+    assertEquals(1, s.count(LongPoint.newSetQuery("float", 17)));
+    assertEquals(3, s.count(LongPoint.newSetQuery("float", 17, 97, 42)));
+    assertEquals(3, s.count(LongPoint.newSetQuery("float", -7, 17, 42, 97)));
+    assertEquals(3, s.count(LongPoint.newSetQuery("float", 17, 20, 42, 97)));
+    assertEquals(3, s.count(LongPoint.newSetQuery("float", 17, 105, 42, 97)));
+
+    assertEquals(0, s.count(LongPoint.newSetQuery("double", 16)));
+    assertEquals(1, s.count(LongPoint.newSetQuery("double", 17)));
+    assertEquals(3, s.count(LongPoint.newSetQuery("double", 17, 97, 42)));
+    assertEquals(3, s.count(LongPoint.newSetQuery("double", -7, 17, 42, 97)));
+    assertEquals(3, s.count(LongPoint.newSetQuery("double", 17, 20, 42, 97)));
+    assertEquals(3, s.count(LongPoint.newSetQuery("double", 17, 105, 42, 97)));
+
     w.close();
     r.close();
     dir.close();
@@ -1313,8 +1333,12 @@ public class TestPointQueries extends LuceneTestCase {
     Document doc = new Document();
     doc.add(new IntPoint("int", 17));
     doc.add(new IntPoint("int", 42));
-    doc.add(new LongPoint("long", 17));
-    doc.add(new LongPoint("long", 42));
+    doc.add(new LongPoint("long", 17L));
+    doc.add(new LongPoint("long", 42L));
+    doc.add(new FloatPoint("float", 17.0f));
+    doc.add(new FloatPoint("float", 42.0f));
+    doc.add(new DoublePoint("double", 17.0));
+    doc.add(new DoublePoint("double", 42.0));
     w.addDocument(doc);
 
     IndexReader r = DirectoryReader.open(w);
@@ -1331,6 +1355,18 @@ public class TestPointQueries extends LuceneTestCase {
     assertEquals(1, s.count(LongPoint.newSetQuery("long", -7, 17, 42, 97)));
     assertEquals(0, s.count(LongPoint.newSetQuery("long", 16, 20, 41, 97)));
 
+    assertEquals(0, s.count(FloatPoint.newSetQuery("float", 16)));
+    assertEquals(1, s.count(FloatPoint.newSetQuery("float", 17)));
+    assertEquals(1, s.count(FloatPoint.newSetQuery("float", 17, 97, 42)));
+    assertEquals(1, s.count(FloatPoint.newSetQuery("float", -7, 17, 42, 97)));
+    assertEquals(0, s.count(FloatPoint.newSetQuery("float", 16, 20, 41, 97)));
+
+    assertEquals(0, s.count(DoublePoint.newSetQuery("double", 16)));
+    assertEquals(1, s.count(DoublePoint.newSetQuery("double", 17)));
+    assertEquals(1, s.count(DoublePoint.newSetQuery("double", 17, 97, 42)));
+    assertEquals(1, s.count(DoublePoint.newSetQuery("double", -7, 17, 42, 97)));
+    assertEquals(0, s.count(DoublePoint.newSetQuery("double", 16, 20, 41, 97)));
+
     w.close();
     r.close();
     dir.close();
@@ -1350,7 +1386,9 @@ public class TestPointQueries extends LuceneTestCase {
       }
       Document doc = new Document();
       doc.add(new IntPoint("int", x));
-      doc.add(new LongPoint("long", x));
+      doc.add(new LongPoint("long", (long) x));
+      doc.add(new FloatPoint("float", (float) x));
+      doc.add(new DoublePoint("double", (double) x));
       w.addDocument(doc);
     }
 
@@ -1367,6 +1405,19 @@ public class TestPointQueries extends LuceneTestCase {
     assertEquals(zeroCount, s.count(LongPoint.newSetQuery("long", 7, 0)));
     assertEquals(10000-zeroCount, s.count(LongPoint.newSetQuery("long", 1)));
     assertEquals(0, s.count(LongPoint.newSetQuery("long", 2)));
+
+    assertEquals(zeroCount, s.count(FloatPoint.newSetQuery("float", 0)));
+    assertEquals(zeroCount, s.count(FloatPoint.newSetQuery("float", 0, -7)));
+    assertEquals(zeroCount, s.count(FloatPoint.newSetQuery("float", 7, 0)));
+    assertEquals(10000-zeroCount, s.count(FloatPoint.newSetQuery("float", 1)));
+    assertEquals(0, s.count(FloatPoint.newSetQuery("float", 2)));
+
+    assertEquals(zeroCount, s.count(DoublePoint.newSetQuery("double", 0)));
+    assertEquals(zeroCount, s.count(DoublePoint.newSetQuery("double", 0, -7)));
+    assertEquals(zeroCount, s.count(DoublePoint.newSetQuery("double", 7, 0)));
+    assertEquals(10000-zeroCount, s.count(DoublePoint.newSetQuery("double", 1)));
+    assertEquals(0, s.count(DoublePoint.newSetQuery("double", 2)));
+
     w.close();
     r.close();
     dir.close();
@@ -1386,7 +1437,9 @@ public class TestPointQueries extends LuceneTestCase {
       }
       Document doc = new Document();
       doc.add(new IntPoint("int", x));
-      doc.add(new LongPoint("long", x));
+      doc.add(new LongPoint("long", (long) x));
+      doc.add(new FloatPoint("float", (float) x));
+      doc.add(new DoublePoint("double", (double) x));
       w.addDocument(doc);
     }
 
@@ -1403,6 +1456,19 @@ public class TestPointQueries extends LuceneTestCase {
     assertEquals(zeroCount, s.count(LongPoint.newSetQuery("long", 7, 0)));
     assertEquals(10000-zeroCount, s.count(LongPoint.newSetQuery("long", 200)));
     assertEquals(0, s.count(LongPoint.newSetQuery("long", 2)));
+
+    assertEquals(zeroCount, s.count(FloatPoint.newSetQuery("float", 0)));
+    assertEquals(zeroCount, s.count(FloatPoint.newSetQuery("float", 0, -7)));
+    assertEquals(zeroCount, s.count(FloatPoint.newSetQuery("float", 7, 0)));
+    assertEquals(10000-zeroCount, s.count(FloatPoint.newSetQuery("float", 200)));
+    assertEquals(0, s.count(FloatPoint.newSetQuery("float", 2)));
+
+    assertEquals(zeroCount, s.count(DoublePoint.newSetQuery("double", 0)));
+    assertEquals(zeroCount, s.count(DoublePoint.newSetQuery("double", 0, -7)));
+    assertEquals(zeroCount, s.count(DoublePoint.newSetQuery("double", 7, 0)));
+    assertEquals(10000-zeroCount, s.count(DoublePoint.newSetQuery("double", 200)));
+    assertEquals(0, s.count(DoublePoint.newSetQuery("double", 2)));
+
     w.close();
     r.close();
     dir.close();
@@ -1428,5 +1494,11 @@ public class TestPointQueries extends LuceneTestCase {
 
     // long
     assertEquals("long:{-42 18}", LongPoint.newSetQuery("long", -42L, 18L).toString());
+
+    // float
+    assertEquals("float:{-42.0 18.0}", FloatPoint.newSetQuery("float", -42.0f, 18.0f).toString());
+
+    // double
+    assertEquals("double:{-42.0 18.0}", DoublePoint.newSetQuery("double", -42.0, 18.0).toString());
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/30fcafdf/lucene/sandbox/src/java/org/apache/lucene/document/BigIntegerPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/BigIntegerPoint.java b/lucene/sandbox/src/java/org/apache/lucene/document/BigIntegerPoint.java
index f88c85a..6e1c6ac 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/BigIntegerPoint.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/BigIntegerPoint.java
@@ -212,4 +212,6 @@ public class BigIntegerPoint extends Field {
       }
     };
   }
+
+  // nocommit newSetQuery
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/30fcafdf/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressPoint.java b/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressPoint.java
index 51ada8f..10abfdd 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressPoint.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressPoint.java
@@ -205,4 +205,6 @@ public class InetAddressPoint extends Field {
       }
     };
   }
+
+  // nocommit newSetQuery
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/30fcafdf/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 539987c..9e18c19 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java
@@ -83,4 +83,6 @@ public class LatLonPoint extends Field {
   public static double decodeLon(int x) {
     return x / LON_SCALE;
   }
+
+  // nocommit newSetQuery
 }


[04/18] lucene-solr git commit: Merge branch 'master' into point_set_query

Posted by mi...@apache.org.
Merge branch 'master' into point_set_query

Conflicts:
	lucene/core/src/java/org/apache/lucene/document/IntPoint.java


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

Branch: refs/heads/master
Commit: 8a4c19777e46de93fe6021cc3c75961695c720ed
Parents: 1654818 099e031
Author: Mike McCandless <mi...@apache.org>
Authored: Tue Feb 23 17:15:20 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Tue Feb 23 17:15:20 2016 -0500

----------------------------------------------------------------------
 .../org/apache/lucene/document/BinaryPoint.java | 101 ++++-
 .../org/apache/lucene/document/DoublePoint.java |  99 ++++-
 .../org/apache/lucene/document/FloatPoint.java  |  99 ++++-
 .../org/apache/lucene/document/IntPoint.java    | 172 ++++++--
 .../org/apache/lucene/document/LongPoint.java   | 101 ++++-
 .../apache/lucene/search/PointRangeQuery.java   | 402 ++-----------------
 .../index/TestDemoParallelLeafReader.java       |   3 +-
 .../apache/lucene/search/TestPointQueries.java  | 104 ++---
 .../TestUsageTrackingFilterCachingPolicy.java   |   4 +-
 .../demo/facet/DistanceFacetsExample.java       |  10 +-
 .../lucene/demo/facet/RangeFacetsExample.java   |   4 +-
 .../facet/range/TestRangeFacetCounts.java       |  21 +-
 .../search/highlight/HighlighterTest.java       |   3 +-
 .../apache/lucene/document/BigIntegerPoint.java |  56 +--
 .../lucene/document/InetAddressPoint.java       |  64 +--
 .../lucene/document/TestBigIntegerPoint.java    |   8 +-
 .../lucene/document/TestInetAddressPoint.java   |  20 +-
 .../lucene/search/TestDocValuesRangeQuery.java  |   9 +-
 .../suggest/document/TestSuggestField.java      |   3 +-
 .../lucene/index/BasePointFormatTestCase.java   |   5 +-
 20 files changed, 678 insertions(+), 610 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8a4c1977/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
----------------------------------------------------------------------
diff --cc lucene/core/src/java/org/apache/lucene/document/IntPoint.java
index 42091d9,177ae99..d4bb28a
--- a/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
@@@ -16,19 -16,23 +16,28 @@@
   */
  package org.apache.lucene.document;
  
 +import java.io.IOException;
 +import java.util.Arrays;
 +
 +import org.apache.lucene.search.PointInSetQuery;
+ import org.apache.lucene.search.PointRangeQuery;
  import org.apache.lucene.util.BytesRef;
 +import org.apache.lucene.util.BytesRefIterator;
  import org.apache.lucene.util.NumericUtils;
  
- /** An int field that is indexed dimensionally such that finding
-  *  all documents within an N-dimensional shape or range at search time is
-  *  efficient.  Multiple values for the same field in one documents
-  *  is allowed. */
- 
+ /** 
+  * An int field that is indexed dimensionally such that finding
+  * all documents within an N-dimensional shape or range at search time is
+  * efficient.  Multiple values for the same field in one documents
+  * is allowed.
+  * <p>
+  * This field defines static factory methods for creating common queries:
+  * <ul>
+  *   <li>{@link #newExactQuery newExactQuery()} for matching an exact 1D point.
+  *   <li>{@link #newRangeQuery newRangeQuery()} for matching a 1D range.
+  *   <li>{@link #newMultiRangeQuery newMultiRangeQuery()} for matching points/ranges in n-dimensional space.
+  * </ul>
+  */
  public final class IntPoint extends Field {
  
    private static FieldType getType(int numDims) {
@@@ -171,4 -140,76 +145,116 @@@
    public static Integer decodeDimension(byte value[], int offset) {
      return NumericUtils.bytesToInt(value, offset);
    }
+   
+   // static methods for generating queries
+   
+   /** 
+    * Create a query for matching an exact integer value.
+    * <p>
+    * This is for simple one-dimension points, for multidimensional points use
+    * {@link #newMultiRangeQuery newMultiRangeQuery()} instead.
+    *
+    * @param field field name. must not be {@code null}.
+    * @param value exact value
+    * @throws IllegalArgumentException if {@code field} is null.
+    * @return a query matching documents with this exact value
+    */
+   public static PointRangeQuery newExactQuery(String field, int value) {
+     return newRangeQuery(field, value, true, value, true);
+   }
+ 
+   /** 
+    * Create a range query for integer values.
+    * <p>
+    * This is for simple one-dimension ranges, for multidimensional ranges use
+    * {@link #newMultiRangeQuery newMultiRangeQuery()} instead.
+    * <p>
+    * You can have half-open ranges (which are in fact &lt;/&le; or &gt;/&ge; queries)
+    * by setting the {@code lowerValue} or {@code upperValue} to {@code null}. 
+    * <p>
+    * By setting inclusive ({@code lowerInclusive} or {@code upperInclusive}) to false, it will
+    * match all documents excluding the bounds, with inclusive on, the boundaries are hits, too.
+    *
+    * @param field field name. must not be {@code null}.
+    * @param lowerValue lower portion of the range. {@code null} means "open".
+    * @param lowerInclusive {@code true} if the lower portion of the range is inclusive, {@code false} if it should be excluded.
+    * @param upperValue upper portion of the range. {@code null} means "open".
+    * @param upperInclusive {@code true} if the upper portion of the range is inclusive, {@code false} if it should be excluded.
+    * @throws IllegalArgumentException if {@code field} is null.
+    * @return a query matching documents within this range.
+    */
+   public static PointRangeQuery newRangeQuery(String field, Integer lowerValue, boolean lowerInclusive, Integer upperValue, boolean upperInclusive) {
+     return newMultiRangeQuery(field, 
+                               new Integer[] { lowerValue },
+                               new boolean[] { lowerInclusive }, 
+                               new Integer[] { upperValue },
+                               new boolean[] { upperInclusive });
+   }
+ 
+   /** 
+    * Create a multidimensional range query for integer values.
+    * <p>
+    * You can have half-open ranges (which are in fact &lt;/&le; or &gt;/&ge; queries)
+    * by setting a {@code lowerValue} element or {@code upperValue} element to {@code null}. 
+    * <p>
+    * By setting a dimension's inclusive ({@code lowerInclusive} or {@code upperInclusive}) to false, it will
+    * match all documents excluding the bounds, with inclusive on, the boundaries are hits, too.
+    *
+    * @param field field name. must not be {@code null}.
+    * @param lowerValue lower portion of the range. {@code null} values mean "open" for that dimension.
+    * @param lowerInclusive {@code true} if the lower portion of the range is inclusive, {@code false} if it should be excluded.
+    * @param upperValue upper portion of the range. {@code null} values mean "open" for that dimension.
+    * @param upperInclusive {@code true} if the upper portion of the range is inclusive, {@code false} if it should be excluded.
+    * @throws IllegalArgumentException if {@code field} is null, or if {@code lowerValue.length != upperValue.length}
+    * @return a query matching documents within this range.
+    */
+   public static PointRangeQuery newMultiRangeQuery(String field, Integer[] lowerValue, boolean lowerInclusive[], Integer[] upperValue, boolean upperInclusive[]) {
+     PointRangeQuery.checkArgs(field, lowerValue, upperValue);
+     return new PointRangeQuery(field, IntPoint.encode(lowerValue), lowerInclusive, IntPoint.encode(upperValue), upperInclusive) {
+       @Override
+       protected String toString(byte[] value) {
+         return IntPoint.decodeDimension(value, 0).toString();
+       }
+     };
+   }
++
++  /**
++   * Returns a query efficiently finding all documents indexed with any of the specified 1D values.
++   * 
++   * @param field field name. must not be {@code null}.
++   * @param valuesIn all int values to search for
++   */
++  public static PointInSetQuery newSetQuery(String field, int... valuesIn) throws IOException {
++
++    // Don't unexpectedly change the user's incoming array:
++    int[] values = valuesIn.clone();
++
++    Arrays.sort(values);
++
++    final BytesRef value = new BytesRef(new byte[Integer.BYTES]);
++    value.length = Integer.BYTES;
++
++    return new PointInSetQuery(field, 1, Integer.BYTES,
++                               new BytesRefIterator() {
++
++                                 int upto;
++
++                                 @Override
++                                 public BytesRef next() {
++                                   if (upto == values.length) {
++                                     return null;
++                                   } else {
++                                     IntPoint.encodeDimension(values[upto], value.bytes, 0);
++                                     upto++;
++                                     return value;
++                                   }
++                                 }
++                               }) {
++      @Override
++      protected String toString(byte[] value) {
++        assert value.length == Integer.BYTES;
++        return Integer.toString(decodeDimension(value, 0));
++      }
++    };
++  }
  }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8a4c1977/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
----------------------------------------------------------------------


[06/18] lucene-solr git commit: address some nocommits; add randomized test for 1D ints

Posted by mi...@apache.org.
address some nocommits; add randomized test for 1D ints


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

Branch: refs/heads/master
Commit: 3f5ed6eb674c535f14682da017270d768595e0f4
Parents: 8d88bb7
Author: Mike McCandless <mi...@apache.org>
Authored: Tue Feb 23 18:41:03 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Tue Feb 23 18:41:03 2016 -0500

----------------------------------------------------------------------
 .../apache/lucene/search/PointInSetQuery.java   |  13 +-
 .../apache/lucene/search/TestPointQueries.java  | 140 +++++++++++++++++++
 2 files changed, 148 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3f5ed6eb/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java b/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
index adb9c53..d0a232b 100644
--- a/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
@@ -42,7 +42,6 @@ import org.apache.lucene.util.StringHelper;
 /** Finds all documents whose point value, previously indexed with e.g. {@link org.apache.lucene.document.LongPoint}, is contained in the
  *  specified set */
 
-// nocommit make abstract
 public class PointInSetQuery extends Query {
   // A little bit overkill for us, since all of our "terms" are always in the same field:
   final PrefixCodedTerms sortedPackedPoints;
@@ -54,8 +53,13 @@ public class PointInSetQuery extends Query {
   /** {@code packedPoints} must already be sorted! */
   protected PointInSetQuery(String field, int numDims, int bytesPerDim, BytesRefIterator packedPoints) throws IOException {
     this.field = field;
-    // nocommit validate these:
+    if (bytesPerDim < 1 || bytesPerDim > PointValues.MAX_NUM_BYTES) {
+      throw new IllegalArgumentException("bytesPerDim must be > 0 and <= " + PointValues.MAX_NUM_BYTES + "; got " + bytesPerDim);
+    }
     this.bytesPerDim = bytesPerDim;
+    if (numDims < 1 || bytesPerDim > PointValues.MAX_DIMENSIONS) {
+      throw new IllegalArgumentException("numDims must be > 0 and <= " + PointValues.MAX_DIMENSIONS + "; got " + numDims);
+    }
     this.numDims = numDims;
 
     // In the 1D case this works well (the more points, the more common prefixes they share, typically), but in
@@ -64,9 +68,8 @@ public class PointInSetQuery extends Query {
     BytesRefBuilder previous = null;
     BytesRef current;
     while ((current = packedPoints.next()) != null) {
-      // nocommit make sure a test tests this:
       if (current.length != numDims * bytesPerDim) {
-        throw new IllegalArgumentException("packed point length should be " + (numDims * bytesPerDim) + " but got " + current.length + "; field=\"" + field + "\", numDims=" + numDims + " bytesPerDim=" + bytesPerDim);
+        throw new IllegalArgumentException("packed point length should be " + (numDims * bytesPerDim) + " but got " + current.length + "; field=\"" + field + "\" numDims=" + numDims + " bytesPerDim=" + bytesPerDim);
       }
       if (previous == null) {
         previous = new BytesRefBuilder();
@@ -104,7 +107,7 @@ public class PointInSetQuery extends Query {
         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()) {
+        if (fieldInfo.getPointNumBytes() != bytesPerDim) {
           throw new IllegalArgumentException("field=\"" + field + "\" was indexed with bytesPerDim=" + fieldInfo.getPointNumBytes() + " but this query has bytesPerDim=" + bytesPerDim);
         }
         int bytesPerDim = fieldInfo.getPointNumBytes();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3f5ed6eb/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 5bc2fdb..7e8eefd 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
@@ -22,7 +22,9 @@ import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.BitSet;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicBoolean;
 
@@ -57,6 +59,7 @@ import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefIterator;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.NumericUtils;
@@ -1115,6 +1118,129 @@ public class TestPointQueries extends LuceneTestCase {
 
   }
 
+  private int[] toArray(Set<Integer> valuesSet) {
+    int[] values = new int[valuesSet.size()];
+    int upto = 0;
+    for(Integer value : valuesSet) {
+      values[upto++] = value;
+    }
+    return values;
+  }
+
+  public void testRandomPointInSetQuery() throws Exception {
+    final Set<Integer> valuesSet = new HashSet<>();
+    int numValues = TestUtil.nextInt(random(), 1, 100);
+    while (valuesSet.size() < numValues) {
+      valuesSet.add(random().nextInt());
+    }
+    int[] values = toArray(valuesSet);
+    int numDocs = TestUtil.nextInt(random(), 1, 10000);
+
+    if (VERBOSE) {
+      System.out.println("TEST: numValues=" + numValues + " numDocs=" + numDocs);
+    }
+
+    Directory dir;
+    if (numDocs > 100000) {
+      dir = newFSDirectory(createTempDir("TestPointQueries"));
+    } else {
+      dir = newDirectory();
+    }
+
+    IndexWriterConfig iwc = newIndexWriterConfig();
+    iwc.setCodec(getCodec());
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
+
+    // nocommit multi-valued too
+
+    int[] docValues = new int[numDocs];
+    for(int i=0;i<numDocs;i++) {
+      int x = values[random().nextInt(values.length)];
+      Document doc = new Document();
+      doc.add(new IntPoint("int", x));
+      docValues[i] = x;
+      w.addDocument(doc);
+    }
+
+    if (random().nextBoolean()) {
+      if (VERBOSE) {
+        System.out.println("  forceMerge(1)");
+      }
+      w.forceMerge(1);
+    }
+    final IndexReader r = w.getReader();
+    w.close();
+
+    IndexSearcher s = newSearcher(r);
+
+    int numThreads = TestUtil.nextInt(random(), 2, 5);
+
+    if (VERBOSE) {
+      System.out.println("TEST: use " + numThreads + " query threads; searcher=" + s);
+    }
+
+    List<Thread> threads = new ArrayList<>();
+    final int iters = atLeast(100);
+
+    final CountDownLatch startingGun = new CountDownLatch(1);
+    final AtomicBoolean failed = new AtomicBoolean();
+
+    for(int i=0;i<numThreads;i++) {
+      Thread thread = new Thread() {
+          @Override
+          public void run() {
+            try {
+              _run();
+            } catch (Exception e) {
+              failed.set(true);
+              throw new RuntimeException(e);
+            }
+          }
+
+          private void _run() throws Exception {
+            startingGun.await();
+
+            for (int iter=0;iter<iters && failed.get() == false;iter++) {
+
+              int numValidValuesToQuery = random().nextInt(values.length);
+
+              Set<Integer> valuesToQuery = new HashSet<>();
+              while (valuesToQuery.size() < numValidValuesToQuery) {
+                valuesToQuery.add(values[random().nextInt(values.length)]);
+              }
+
+              int numExtraValuesToQuery = random().nextInt(20);
+              while (valuesToQuery.size() < numValidValuesToQuery + numExtraValuesToQuery) {
+                // nocommit fix test to sometimes use "narrow" range of values
+                valuesToQuery.add(random().nextInt());
+              }
+
+              int expectedCount = 0;
+              for(int value : docValues) {
+                if (valuesToQuery.contains(value)) {
+                  expectedCount++;
+                }
+              }
+
+              if (VERBOSE) {
+                System.out.println("TEST: thread=" + Thread.currentThread() + " values=" + valuesToQuery + " expectedCount=" + expectedCount);
+              }
+
+              assertEquals(expectedCount, s.count(IntPoint.newSetQuery("int", toArray(valuesToQuery))));
+            }
+          }
+        };
+      thread.setName("T" + i);
+      thread.start();
+      threads.add(thread);
+    }
+    startingGun.countDown();
+    for(Thread thread : threads) {
+      thread.join();
+    }
+    IOUtils.close(r, dir);
+  }
+
   // nocommit fix existing randomized tests to sometimes randomly use PointInSet instead
 
   // nocommit need 2D test too
@@ -1207,4 +1333,18 @@ public class TestPointQueries extends LuceneTestCase {
     r.close();
     dir.close();
   }
+
+  public void testInvalidPointInSetQuery() throws Exception {
+    IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+                                                     () -> {
+                                                       new PointInSetQuery("foo", 3, 4,
+                                                                           new BytesRefIterator() {
+                                                                             @Override
+                                                                             public BytesRef next() {
+                                                                               return new BytesRef(new byte[3]);
+                                                                             }
+                                                                           });
+                                                     });
+    assertEquals("packed point length should be 12 but got 3; field=\"foo\" numDims=3 bytesPerDim=4", expected.getMessage());
+  }
 }


[07/18] lucene-solr git commit: add LongPoint.newSetQuery; improve PointInSetQuery.toString

Posted by mi...@apache.org.
add LongPoint.newSetQuery; improve PointInSetQuery.toString


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

Branch: refs/heads/master
Commit: 79c8bce1af49277666d78bd4259aa47b56e44c7b
Parents: 3f5ed6e
Author: Mike McCandless <mi...@apache.org>
Authored: Wed Feb 24 04:50:37 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Wed Feb 24 04:50:37 2016 -0500

----------------------------------------------------------------------
 .../org/apache/lucene/document/IntPoint.java    |  6 +-
 .../org/apache/lucene/document/LongPoint.java   | 45 ++++++++++
 .../apache/lucene/search/PointInSetQuery.java   | 13 +--
 .../apache/lucene/search/TestPointQueries.java  | 94 ++++++++++++++++++--
 4 files changed, 143 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/79c8bce1/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/IntPoint.java b/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
index d4bb28a..2b255d7 100644
--- a/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
@@ -219,14 +219,14 @@ public final class IntPoint extends Field {
   }
 
   /**
-   * Returns a query efficiently finding all documents indexed with any of the specified 1D values.
+   * Create a query matching any of the specified 1D values.  This is the points equivalent of {@code TermsQuery}.
    * 
    * @param field field name. must not be {@code null}.
-   * @param valuesIn all int values to search for
+   * @param valuesIn all int values to match
    */
   public static PointInSetQuery newSetQuery(String field, int... valuesIn) throws IOException {
 
-    // Don't unexpectedly change the user's incoming array:
+    // Don't unexpectedly change the user's incoming values array:
     int[] values = valuesIn.clone();
 
     Arrays.sort(values);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/79c8bce1/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/LongPoint.java b/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
index d3f8c17..16f7e6c 100644
--- a/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
@@ -16,8 +16,13 @@
  */
 package org.apache.lucene.document;
 
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.lucene.search.PointInSetQuery;
 import org.apache.lucene.search.PointRangeQuery;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefIterator;
 import org.apache.lucene.util.NumericUtils;
 
 /** 
@@ -212,4 +217,44 @@ public final class LongPoint extends Field {
       }
     };
   }
+
+  /**
+   * Create a query matching any of the specified 1D values.  This is the points equivalent of {@code TermsQuery}.
+   * 
+   * @param field field name. must not be {@code null}.
+   * @param valuesIn all int values to match
+   */
+  public static PointInSetQuery newSetQuery(String field, long... valuesIn) throws IOException {
+
+    // Don't unexpectedly change the user's incoming values array:
+    long[] values = valuesIn.clone();
+
+    Arrays.sort(values);
+
+    final BytesRef value = new BytesRef(new byte[Long.BYTES]);
+    value.length = Long.BYTES;
+
+    return new PointInSetQuery(field, 1, Long.BYTES,
+                               new BytesRefIterator() {
+
+                                 int upto;
+
+                                 @Override
+                                 public BytesRef next() {
+                                   if (upto == values.length) {
+                                     return null;
+                                   } else {
+                                     LongPoint.encodeDimension(values[upto], value.bytes, 0);
+                                     upto++;
+                                     return value;
+                                   }
+                                 }
+                               }) {
+      @Override
+      protected String toString(byte[] value) {
+        assert value.length == Long.BYTES;
+        return Long.toString(decodeDimension(value, 0));
+      }
+    };
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/79c8bce1/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java b/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
index d0a232b..df50c9e 100644
--- a/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
@@ -320,24 +320,25 @@ public class PointInSetQuery extends Query {
   @Override
   public String toString(String field) {
     final StringBuilder sb = new StringBuilder();
-    sb.append(getClass().getSimpleName());
-    sb.append(':');
     if (this.field.equals(field) == false) {
-      sb.append(" field=");
       sb.append(this.field);
       sb.append(':');
     }
 
-    sb.append(" points:");
+    sb.append("{");
 
     TermIterator iterator = sortedPackedPoints.iterator();
     byte[] pointBytes = new byte[numDims * bytesPerDim];
+    boolean first = true;
     for (BytesRef point = iterator.next(); point != null; point = iterator.next()) {
-      sb.append(' ');
+      if (first == false) {
+        sb.append(" ");
+      }
+      first = false;
       System.arraycopy(point.bytes, point.offset, pointBytes, 0, pointBytes.length);
       sb.append(toString(pointBytes));
     }
-
+    sb.append("}");
     return sb.toString();
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/79c8bce1/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 7e8eefd..843a929 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
@@ -1127,11 +1127,31 @@ public class TestPointQueries extends LuceneTestCase {
     return values;
   }
 
+  private static int randomIntValue(Integer min, Integer max) {
+    if (min == null) {
+      return random().nextInt();
+    } else {
+      return TestUtil.nextInt(random(), min, max);
+    }
+  }
+
   public void testRandomPointInSetQuery() throws Exception {
+
+    boolean useNarrowRange = random().nextBoolean();
+    final Integer valueMin;
+    final Integer valueMax;
+    if (useNarrowRange) {
+      int gap = random().nextInt(100);
+      valueMin = random().nextInt(Integer.MAX_VALUE-gap);
+      valueMax = valueMin + gap;
+    } else {
+      valueMin = null;
+      valueMax = null;
+    }
     final Set<Integer> valuesSet = new HashSet<>();
     int numValues = TestUtil.nextInt(random(), 1, 100);
     while (valuesSet.size() < numValues) {
-      valuesSet.add(random().nextInt());
+      valuesSet.add(randomIntValue(valueMin, valueMax));
     }
     int[] values = toArray(valuesSet);
     int numDocs = TestUtil.nextInt(random(), 1, 10000);
@@ -1211,8 +1231,7 @@ public class TestPointQueries extends LuceneTestCase {
 
               int numExtraValuesToQuery = random().nextInt(20);
               while (valuesToQuery.size() < numValidValuesToQuery + numExtraValuesToQuery) {
-                // nocommit fix test to sometimes use "narrow" range of values
-                valuesToQuery.add(random().nextInt());
+                valuesToQuery.add(randomIntValue(valueMin, valueMax));
               }
 
               int expectedCount = 0;
@@ -1241,8 +1260,6 @@ public class TestPointQueries extends LuceneTestCase {
     IOUtils.close(r, dir);
   }
 
-  // nocommit fix existing randomized tests to sometimes randomly use PointInSet instead
-
   // nocommit need 2D test too
 
   public void testBasicPointInSetQuery() throws Exception {
@@ -1253,14 +1270,17 @@ public class TestPointQueries extends LuceneTestCase {
 
     Document doc = new Document();
     doc.add(new IntPoint("int", 17));
+    doc.add(new LongPoint("long", 17));
     w.addDocument(doc);
 
     doc = new Document();
     doc.add(new IntPoint("int", 42));
+    doc.add(new LongPoint("long", 42));
     w.addDocument(doc);
 
     doc = new Document();
     doc.add(new IntPoint("int", 97));
+    doc.add(new LongPoint("long", 97));
     w.addDocument(doc);
 
     IndexReader r = DirectoryReader.open(w);
@@ -1271,6 +1291,46 @@ public class TestPointQueries extends LuceneTestCase {
     assertEquals(3, s.count(IntPoint.newSetQuery("int", -7, 17, 42, 97)));
     assertEquals(3, s.count(IntPoint.newSetQuery("int", 17, 20, 42, 97)));
     assertEquals(3, s.count(IntPoint.newSetQuery("int", 17, 105, 42, 97)));
+
+    assertEquals(0, s.count(LongPoint.newSetQuery("long", 16)));
+    assertEquals(1, s.count(LongPoint.newSetQuery("long", 17)));
+    assertEquals(3, s.count(LongPoint.newSetQuery("long", 17, 97, 42)));
+    assertEquals(3, s.count(LongPoint.newSetQuery("long", -7, 17, 42, 97)));
+    assertEquals(3, s.count(LongPoint.newSetQuery("long", 17, 20, 42, 97)));
+    assertEquals(3, s.count(LongPoint.newSetQuery("long", 17, 105, 42, 97)));
+
+    w.close();
+    r.close();
+    dir.close();
+  }
+
+  public void testBasicMultiValuedPointInSetQuery() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = newIndexWriterConfig();
+    iwc.setCodec(getCodec());
+    IndexWriter w = new IndexWriter(dir, iwc);
+
+    Document doc = new Document();
+    doc.add(new IntPoint("int", 17));
+    doc.add(new IntPoint("int", 42));
+    doc.add(new LongPoint("long", 17));
+    doc.add(new LongPoint("long", 42));
+    w.addDocument(doc);
+
+    IndexReader r = DirectoryReader.open(w);
+    IndexSearcher s = newSearcher(r);
+    assertEquals(0, s.count(IntPoint.newSetQuery("int", 16)));
+    assertEquals(1, s.count(IntPoint.newSetQuery("int", 17)));
+    assertEquals(1, s.count(IntPoint.newSetQuery("int", 17, 97, 42)));
+    assertEquals(1, s.count(IntPoint.newSetQuery("int", -7, 17, 42, 97)));
+    assertEquals(0, s.count(IntPoint.newSetQuery("int", 16, 20, 41, 97)));
+
+    assertEquals(0, s.count(LongPoint.newSetQuery("long", 16)));
+    assertEquals(1, s.count(LongPoint.newSetQuery("long", 17)));
+    assertEquals(1, s.count(LongPoint.newSetQuery("long", 17, 97, 42)));
+    assertEquals(1, s.count(LongPoint.newSetQuery("long", -7, 17, 42, 97)));
+    assertEquals(0, s.count(LongPoint.newSetQuery("long", 16, 20, 41, 97)));
+
     w.close();
     r.close();
     dir.close();
@@ -1290,6 +1350,7 @@ public class TestPointQueries extends LuceneTestCase {
       }
       Document doc = new Document();
       doc.add(new IntPoint("int", x));
+      doc.add(new LongPoint("long", x));
       w.addDocument(doc);
     }
 
@@ -1300,12 +1361,18 @@ public class TestPointQueries extends LuceneTestCase {
     assertEquals(zeroCount, s.count(IntPoint.newSetQuery("int", 7, 0)));
     assertEquals(10000-zeroCount, s.count(IntPoint.newSetQuery("int", 1)));
     assertEquals(0, s.count(IntPoint.newSetQuery("int", 2)));
+
+    assertEquals(zeroCount, s.count(LongPoint.newSetQuery("long", 0)));
+    assertEquals(zeroCount, s.count(LongPoint.newSetQuery("long", 0, -7)));
+    assertEquals(zeroCount, s.count(LongPoint.newSetQuery("long", 7, 0)));
+    assertEquals(10000-zeroCount, s.count(LongPoint.newSetQuery("long", 1)));
+    assertEquals(0, s.count(LongPoint.newSetQuery("long", 2)));
     w.close();
     r.close();
     dir.close();
   }
 
-  public void testPointInSetQueryManyEqualValuesBigGap() throws Exception {
+  public void testPointInSetQueryManyEqualValuesWithBigGap() throws Exception {
     Directory dir = newDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig();
     iwc.setCodec(getCodec());
@@ -1319,6 +1386,7 @@ public class TestPointQueries extends LuceneTestCase {
       }
       Document doc = new Document();
       doc.add(new IntPoint("int", x));
+      doc.add(new LongPoint("long", x));
       w.addDocument(doc);
     }
 
@@ -1329,6 +1397,12 @@ public class TestPointQueries extends LuceneTestCase {
     assertEquals(zeroCount, s.count(IntPoint.newSetQuery("int", 7, 0)));
     assertEquals(10000-zeroCount, s.count(IntPoint.newSetQuery("int", 200)));
     assertEquals(0, s.count(IntPoint.newSetQuery("int", 2)));
+
+    assertEquals(zeroCount, s.count(LongPoint.newSetQuery("long", 0)));
+    assertEquals(zeroCount, s.count(LongPoint.newSetQuery("long", 0, -7)));
+    assertEquals(zeroCount, s.count(LongPoint.newSetQuery("long", 7, 0)));
+    assertEquals(10000-zeroCount, s.count(LongPoint.newSetQuery("long", 200)));
+    assertEquals(0, s.count(LongPoint.newSetQuery("long", 2)));
     w.close();
     r.close();
     dir.close();
@@ -1347,4 +1421,12 @@ public class TestPointQueries extends LuceneTestCase {
                                                      });
     assertEquals("packed point length should be 12 but got 3; field=\"foo\" numDims=3 bytesPerDim=4", expected.getMessage());
   }
+
+  public void testPointInSetQueryToString() throws Exception {
+    // int
+    assertEquals("int:{-42 18}", IntPoint.newSetQuery("int", -42, 18).toString());
+
+    // long
+    assertEquals("long:{-42 18}", LongPoint.newSetQuery("long", -42L, 18L).toString());
+  }
 }


[14/18] lucene-solr git commit: add BinaryPoint.newSetQuery

Posted by mi...@apache.org.
add BinaryPoint.newSetQuery


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

Branch: refs/heads/master
Commit: 41a336f1e591c65fc5a6e8c85530367661e8907d
Parents: c056f4b
Author: Mike McCandless <mi...@apache.org>
Authored: Wed Feb 24 17:45:56 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Wed Feb 24 17:45:56 2016 -0500

----------------------------------------------------------------------
 .../org/apache/lucene/document/BinaryPoint.java | 68 ++++++++++++++++++++
 .../apache/lucene/search/PointInSetQuery.java   |  1 -
 .../apache/lucene/search/TestPointQueries.java  | 68 ++++++++++++++++----
 3 files changed, 124 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/41a336f1/lucene/core/src/java/org/apache/lucene/document/BinaryPoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/BinaryPoint.java b/lucene/core/src/java/org/apache/lucene/document/BinaryPoint.java
index 6a3cb85..b8cbc1e 100644
--- a/lucene/core/src/java/org/apache/lucene/document/BinaryPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/BinaryPoint.java
@@ -16,9 +16,16 @@
  */
 package org.apache.lucene.document;
 
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Comparator;
+
+import org.apache.lucene.search.PointInSetQuery;
 import org.apache.lucene.search.PointRangeQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefIterator;
+import org.apache.lucene.util.StringHelper;
 
 /** 
  * A binary field that is indexed dimensionally such that finding
@@ -31,6 +38,7 @@ import org.apache.lucene.util.BytesRef;
  *   <li>{@link #newExactQuery newExactQuery()} for matching an exact 1D point.
  *   <li>{@link #newRangeQuery newRangeQuery()} for matching a 1D range.
  *   <li>{@link #newMultiRangeQuery newMultiRangeQuery()} for matching points/ranges in n-dimensional space.
+ *   <li>{@link #newSetQuery newSetQuery()} for matching a set of 1D values.
  * </ul> 
  */
 public final class BinaryPoint extends Field {
@@ -197,4 +205,64 @@ public final class BinaryPoint extends Field {
       }
     };
   }
+
+  /**
+   * Create a query matching any of the specified 1D values.  This is the points equivalent of {@code TermsQuery}.
+   * 
+   * @param field field name. must not be {@code null}.
+   * @param valuesIn all values to match
+   */
+  public static Query newSetQuery(String field, byte[]... valuesIn) throws IOException {
+
+    // Make sure all byte[] have the same length
+    int bytesPerDim = -1;
+    for(byte[] value : valuesIn) {
+      if (bytesPerDim == -1) {
+        bytesPerDim = value.length;
+      } else if (value.length != bytesPerDim) {
+        throw new IllegalArgumentException("all byte[] must be the same length, but saw " + bytesPerDim + " and " + value.length);
+      }
+    }
+
+    // Don't unexpectedly change the user's incoming values array:
+    byte[][] values = valuesIn.clone();
+
+    Arrays.sort(values,
+                new Comparator<byte[]>() {
+                  @Override
+                  public int compare(byte[] a, byte[] b) {
+                    return StringHelper.compare(a.length, a, 0, b, 0);
+                  }
+                });
+
+    // Silliness:
+    if (bytesPerDim == -1) {
+      // nocommit make sure this is tested
+      bytesPerDim = 1;
+    }
+
+    final BytesRef value = new BytesRef(new byte[bytesPerDim]);
+    
+    return new PointInSetQuery(field, 1, bytesPerDim,
+                               new BytesRefIterator() {
+
+                                 int upto;
+
+                                 @Override
+                                 public BytesRef next() {
+                                   if (upto == values.length) {
+                                     return null;
+                                   } else {
+                                     value.bytes = values[upto];
+                                     upto++;
+                                     return value;
+                                   }
+                                 }
+                               }) {
+      @Override
+      protected String toString(byte[] value) {
+        return new BytesRef(value).toString();
+      }
+    };
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/41a336f1/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java b/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
index c00973e..d34e3f7 100644
--- a/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
@@ -116,7 +116,6 @@ public class PointInSetQuery extends Query {
         if (fieldInfo.getPointNumBytes() != bytesPerDim) {
           throw new IllegalArgumentException("field=\"" + field + "\" was indexed with bytesPerDim=" + fieldInfo.getPointNumBytes() + " but this query has bytesPerDim=" + bytesPerDim);
         }
-        int bytesPerDim = fieldInfo.getPointNumBytes();
 
         DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc());
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/41a336f1/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 51d914d..095ddc9 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
@@ -1383,6 +1383,7 @@ public class TestPointQueries extends LuceneTestCase {
     doc.add(new LongPoint("long", 17L));
     doc.add(new FloatPoint("float", 17.0f));
     doc.add(new DoublePoint("double", 17.0));
+    doc.add(new BinaryPoint("bytes", new byte[] {0, 17}));
     w.addDocument(doc);
 
     doc = new Document();
@@ -1390,6 +1391,7 @@ public class TestPointQueries extends LuceneTestCase {
     doc.add(new LongPoint("long", 42L));
     doc.add(new FloatPoint("float", 42.0f));
     doc.add(new DoublePoint("double", 42.0));
+    doc.add(new BinaryPoint("bytes", new byte[] {0, 42}));
     w.addDocument(doc);
 
     doc = new Document();
@@ -1397,6 +1399,7 @@ public class TestPointQueries extends LuceneTestCase {
     doc.add(new LongPoint("long", 97L));
     doc.add(new FloatPoint("float", 97.0f));
     doc.add(new DoublePoint("double", 97.0));
+    doc.add(new BinaryPoint("bytes", new byte[] {0, 97}));
     w.addDocument(doc);
 
     IndexReader r = DirectoryReader.open(w);
@@ -1415,19 +1418,27 @@ public class TestPointQueries extends LuceneTestCase {
     assertEquals(3, s.count(LongPoint.newSetQuery("long", 17, 20, 42, 97)));
     assertEquals(3, s.count(LongPoint.newSetQuery("long", 17, 105, 42, 97)));
 
-    assertEquals(0, s.count(LongPoint.newSetQuery("float", 16)));
-    assertEquals(1, s.count(LongPoint.newSetQuery("float", 17)));
-    assertEquals(3, s.count(LongPoint.newSetQuery("float", 17, 97, 42)));
-    assertEquals(3, s.count(LongPoint.newSetQuery("float", -7, 17, 42, 97)));
-    assertEquals(3, s.count(LongPoint.newSetQuery("float", 17, 20, 42, 97)));
-    assertEquals(3, s.count(LongPoint.newSetQuery("float", 17, 105, 42, 97)));
+    assertEquals(0, s.count(FloatPoint.newSetQuery("float", 16)));
+    assertEquals(1, s.count(FloatPoint.newSetQuery("float", 17)));
+    assertEquals(3, s.count(FloatPoint.newSetQuery("float", 17, 97, 42)));
+    assertEquals(3, s.count(FloatPoint.newSetQuery("float", -7, 17, 42, 97)));
+    assertEquals(3, s.count(FloatPoint.newSetQuery("float", 17, 20, 42, 97)));
+    assertEquals(3, s.count(FloatPoint.newSetQuery("float", 17, 105, 42, 97)));
 
-    assertEquals(0, s.count(LongPoint.newSetQuery("double", 16)));
-    assertEquals(1, s.count(LongPoint.newSetQuery("double", 17)));
-    assertEquals(3, s.count(LongPoint.newSetQuery("double", 17, 97, 42)));
-    assertEquals(3, s.count(LongPoint.newSetQuery("double", -7, 17, 42, 97)));
-    assertEquals(3, s.count(LongPoint.newSetQuery("double", 17, 20, 42, 97)));
-    assertEquals(3, s.count(LongPoint.newSetQuery("double", 17, 105, 42, 97)));
+    assertEquals(0, s.count(DoublePoint.newSetQuery("double", 16)));
+    assertEquals(1, s.count(DoublePoint.newSetQuery("double", 17)));
+    assertEquals(3, s.count(DoublePoint.newSetQuery("double", 17, 97, 42)));
+    assertEquals(3, s.count(DoublePoint.newSetQuery("double", -7, 17, 42, 97)));
+    assertEquals(3, s.count(DoublePoint.newSetQuery("double", 17, 20, 42, 97)));
+    assertEquals(3, s.count(DoublePoint.newSetQuery("double", 17, 105, 42, 97)));
+
+    // nocommit make sure invalid bytes length hits iae
+    assertEquals(0, s.count(BinaryPoint.newSetQuery("bytes", new byte[] {0, 16})));
+    assertEquals(1, s.count(BinaryPoint.newSetQuery("bytes", new byte[] {0, 17})));
+    assertEquals(3, s.count(BinaryPoint.newSetQuery("bytes", new byte[] {0, 17}, new byte[] {0, 97}, new byte[] {0, 42})));
+    assertEquals(3, s.count(BinaryPoint.newSetQuery("bytes", new byte[] {0, -7}, new byte[] {0, 17}, new byte[] {0, 42}, new byte[] {0, 97})));
+    assertEquals(3, s.count(BinaryPoint.newSetQuery("bytes", new byte[] {0, 17}, new byte[] {0, 20}, new byte[] {0, 42}, new byte[] {0, 97})));
+    assertEquals(3, s.count(BinaryPoint.newSetQuery("bytes", new byte[] {0, 17}, new byte[] {0, 105}, new byte[] {0, 42}, new byte[] {0, 97})));
 
     w.close();
     r.close();
@@ -1449,6 +1460,8 @@ public class TestPointQueries extends LuceneTestCase {
     doc.add(new FloatPoint("float", 42.0f));
     doc.add(new DoublePoint("double", 17.0));
     doc.add(new DoublePoint("double", 42.0));
+    doc.add(new BinaryPoint("bytes", new byte[] {0, 17}));
+    doc.add(new BinaryPoint("bytes", new byte[] {0, 42}));
     w.addDocument(doc);
 
     IndexReader r = DirectoryReader.open(w);
@@ -1477,6 +1490,12 @@ public class TestPointQueries extends LuceneTestCase {
     assertEquals(1, s.count(DoublePoint.newSetQuery("double", -7, 17, 42, 97)));
     assertEquals(0, s.count(DoublePoint.newSetQuery("double", 16, 20, 41, 97)));
 
+    assertEquals(0, s.count(BinaryPoint.newSetQuery("bytes", new byte[] {0, 16})));
+    assertEquals(1, s.count(BinaryPoint.newSetQuery("bytes", new byte[] {0, 17})));
+    assertEquals(1, s.count(BinaryPoint.newSetQuery("bytes", new byte[] {0, 17}, new byte[] {0, 97}, new byte[] {0, 42})));
+    assertEquals(1, s.count(BinaryPoint.newSetQuery("bytes", new byte[] {0, -7}, new byte[] {0, 17}, new byte[] {0, 42}, new byte[] {0, 97})));
+    assertEquals(0, s.count(BinaryPoint.newSetQuery("bytes", new byte[] {0, 16}, new byte[] {0, 20}, new byte[] {0, 41}, new byte[] {0, 97})));
+
     w.close();
     r.close();
     dir.close();
@@ -1499,6 +1518,7 @@ public class TestPointQueries extends LuceneTestCase {
       doc.add(new LongPoint("long", (long) x));
       doc.add(new FloatPoint("float", (float) x));
       doc.add(new DoublePoint("double", (double) x));
+      doc.add(new BinaryPoint("bytes", new byte[] {(byte) x}));
       w.addDocument(doc);
     }
 
@@ -1528,6 +1548,12 @@ public class TestPointQueries extends LuceneTestCase {
     assertEquals(10000-zeroCount, s.count(DoublePoint.newSetQuery("double", 1)));
     assertEquals(0, s.count(DoublePoint.newSetQuery("double", 2)));
 
+    assertEquals(zeroCount, s.count(BinaryPoint.newSetQuery("bytes", new byte[] {0})));
+    assertEquals(zeroCount, s.count(BinaryPoint.newSetQuery("bytes", new byte[] {0}, new byte[] {-7})));
+    assertEquals(zeroCount, s.count(BinaryPoint.newSetQuery("bytes", new byte[] {7}, new byte[] {0})));
+    assertEquals(10000-zeroCount, s.count(BinaryPoint.newSetQuery("bytes", new byte[] {1})));
+    assertEquals(0, s.count(BinaryPoint.newSetQuery("bytes", new byte[] {2})));
+
     w.close();
     r.close();
     dir.close();
@@ -1550,6 +1576,7 @@ public class TestPointQueries extends LuceneTestCase {
       doc.add(new LongPoint("long", (long) x));
       doc.add(new FloatPoint("float", (float) x));
       doc.add(new DoublePoint("double", (double) x));
+      doc.add(new BinaryPoint("bytes", new byte[] {(byte) x}));
       w.addDocument(doc);
     }
 
@@ -1579,6 +1606,12 @@ public class TestPointQueries extends LuceneTestCase {
     assertEquals(10000-zeroCount, s.count(DoublePoint.newSetQuery("double", 200)));
     assertEquals(0, s.count(DoublePoint.newSetQuery("double", 2)));
 
+    assertEquals(zeroCount, s.count(BinaryPoint.newSetQuery("bytes", new byte[] {0})));
+    assertEquals(zeroCount, s.count(BinaryPoint.newSetQuery("bytes", new byte[] {0}, new byte[] {-7})));
+    assertEquals(zeroCount, s.count(BinaryPoint.newSetQuery("bytes", new byte[] {7}, new byte[] {0})));
+    assertEquals(10000-zeroCount, s.count(BinaryPoint.newSetQuery("bytes", new byte[] {(byte) 200})));
+    assertEquals(0, s.count(BinaryPoint.newSetQuery("bytes", new byte[] {2})));
+
     w.close();
     r.close();
     dir.close();
@@ -1598,6 +1631,14 @@ public class TestPointQueries extends LuceneTestCase {
     assertEquals("packed point length should be 12 but got 3; field=\"foo\" numDims=3 bytesPerDim=4", expected.getMessage());
   }
 
+  public void testInvalidPointInSetBinaryQuery() throws Exception {
+    IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
+                                                     () -> {
+                                                       BinaryPoint.newSetQuery("bytes", new byte[] {2}, new byte[0]);
+                                                     });
+    assertEquals("all byte[] must be the same length, but saw 1 and 0", expected.getMessage());
+  }
+
   public void testPointInSetQueryToString() throws Exception {
     // int
     assertEquals("int:{-42 18}", IntPoint.newSetQuery("int", -42, 18).toString());
@@ -1610,5 +1651,8 @@ public class TestPointQueries extends LuceneTestCase {
 
     // double
     assertEquals("double:{-42.0 18.0}", DoublePoint.newSetQuery("double", -42.0, 18.0).toString());
+
+    // binary
+    assertEquals("bytes:{[12] [2a]}", BinaryPoint.newSetQuery("bytes", new byte[] {42}, new byte[] {18}).toString());
   }
 }


[12/18] lucene-solr git commit: add BigIntegerPoint.newSetQuery; fix Float/DoublePoint to encode/decode correctly

Posted by mi...@apache.org.
add BigIntegerPoint.newSetQuery; fix Float/DoublePoint to encode/decode correctly


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

Branch: refs/heads/master
Commit: ae7092037823ca972c781c41fbd6847473dfbe96
Parents: 1c3d52d
Author: Mike McCandless <mi...@apache.org>
Authored: Wed Feb 24 16:39:13 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Wed Feb 24 16:39:13 2016 -0500

----------------------------------------------------------------------
 .../org/apache/lucene/document/DoublePoint.java |  7 +--
 .../org/apache/lucene/document/FloatPoint.java  | 10 ++--
 .../org/apache/lucene/document/IntPoint.java    |  3 +-
 .../org/apache/lucene/document/LongPoint.java   |  3 +-
 .../apache/lucene/search/PointInSetQuery.java   |  4 +-
 .../org/apache/lucene/util/NumericUtils.java    | 15 ++----
 .../apache/lucene/search/TestPointQueries.java  | 55 ++++++++++++++++++++
 .../apache/lucene/document/BigIntegerPoint.java | 44 +++++++++++++++-
 .../lucene/document/InetAddressPoint.java       | 11 ++--
 .../org/apache/lucene/document/LatLonPoint.java |  2 -
 .../lucene/document/TestBigIntegerPoint.java    | 10 ++++
 .../lucene/document/TestInetAddressPoint.java   |  5 +-
 12 files changed, 137 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ae709203/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java b/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
index b6fc1ce..4a78963 100644
--- a/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
@@ -37,6 +37,7 @@ import org.apache.lucene.util.NumericUtils;
  *   <li>{@link #newExactQuery newExactQuery()} for matching an exact 1D point.
  *   <li>{@link #newRangeQuery newRangeQuery()} for matching a 1D range.
  *   <li>{@link #newMultiRangeQuery newMultiRangeQuery()} for matching points/ranges in n-dimensional space.
+ *   <li>{@link #newSetQuery newSetQuery()} for matching a set of 1D values.
  * </ul> 
  */
 public final class DoublePoint extends Field {
@@ -139,12 +140,12 @@ public final class DoublePoint extends Field {
   
   /** Encode single double dimension */
   public static void encodeDimension(double value, byte dest[], int offset) {
-    NumericUtils.longToBytesDirect(NumericUtils.doubleToSortableLong(value), dest, offset);
+    NumericUtils.longToBytes(NumericUtils.doubleToSortableLong(value), dest, offset);
   }
   
   /** Decode single double dimension */
   public static double decodeDimension(byte value[], int offset) {
-    return NumericUtils.sortableLongToDouble(NumericUtils.bytesToLongDirect(value, offset));
+    return NumericUtils.sortableLongToDouble(NumericUtils.bytesToLong(value, offset));
   }
   
   // static methods for generating queries
@@ -223,7 +224,7 @@ public final class DoublePoint extends Field {
    * Create a query matching any of the specified 1D values.  This is the points equivalent of {@code TermsQuery}.
    * 
    * @param field field name. must not be {@code null}.
-   * @param valuesIn all int values to match
+   * @param valuesIn all values to match
    */
   public static Query newSetQuery(String field, double... valuesIn) throws IOException {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ae709203/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java b/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
index dae88b7..be8ba5a 100644
--- a/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
@@ -37,6 +37,7 @@ import org.apache.lucene.util.NumericUtils;
  *   <li>{@link #newExactQuery newExactQuery()} for matching an exact 1D point.
  *   <li>{@link #newRangeQuery newRangeQuery()} for matching a 1D range.
  *   <li>{@link #newMultiRangeQuery newMultiRangeQuery()} for matching points/ranges in n-dimensional space.
+ *   <li>{@link #newSetQuery newSetQuery()} for matching a set of 1D values.
  * </ul>
  */
 public final class FloatPoint extends Field {
@@ -139,12 +140,12 @@ public final class FloatPoint extends Field {
   
   /** Encode single float dimension */
   public static void encodeDimension(float value, byte dest[], int offset) {
-    NumericUtils.intToBytesDirect(NumericUtils.floatToSortableInt(value), dest, offset);
+    NumericUtils.intToBytes(NumericUtils.floatToSortableInt(value), dest, offset);
   }
   
   /** Decode single float dimension */
   public static float decodeDimension(byte value[], int offset) {
-    return NumericUtils.sortableIntToFloat(NumericUtils.bytesToIntDirect(value, offset));
+    return NumericUtils.sortableIntToFloat(NumericUtils.bytesToInt(value, offset));
   }
   
   // static methods for generating queries
@@ -223,7 +224,7 @@ public final class FloatPoint extends Field {
    * Create a query matching any of the specified 1D values.  This is the points equivalent of {@code TermsQuery}.
    * 
    * @param field field name. must not be {@code null}.
-   * @param valuesIn all int values to match
+   * @param valuesIn all values to match
    */
   public static Query newSetQuery(String field, float... valuesIn) throws IOException {
 
@@ -232,8 +233,6 @@ public final class FloatPoint extends Field {
 
     Arrays.sort(values);
 
-    System.out.println("VALUES: " + Arrays.toString(values));
-
     final BytesRef value = new BytesRef(new byte[Float.BYTES]);
 
     return new PointInSetQuery(field, 1, Float.BYTES,
@@ -248,7 +247,6 @@ public final class FloatPoint extends Field {
                                    } else {
                                      encodeDimension(values[upto], value.bytes, 0);
                                      upto++;
-                                     System.out.println("ret: " + value);
                                      return value;
                                    }
                                  }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ae709203/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/IntPoint.java b/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
index 23b7278..ebcdbc5 100644
--- a/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
@@ -37,6 +37,7 @@ import org.apache.lucene.util.NumericUtils;
  *   <li>{@link #newExactQuery newExactQuery()} for matching an exact 1D point.
  *   <li>{@link #newRangeQuery newRangeQuery()} for matching a 1D range.
  *   <li>{@link #newMultiRangeQuery newMultiRangeQuery()} for matching points/ranges in n-dimensional space.
+ *   <li>{@link #newSetQuery newSetQuery()} for matching a set of 1D values.
  * </ul>
  */
 public final class IntPoint extends Field {
@@ -223,7 +224,7 @@ public final class IntPoint extends Field {
    * Create a query matching any of the specified 1D values.  This is the points equivalent of {@code TermsQuery}.
    * 
    * @param field field name. must not be {@code null}.
-   * @param valuesIn all int values to match
+   * @param valuesIn all values to match
    */
   public static Query newSetQuery(String field, int... valuesIn) throws IOException {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ae709203/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/LongPoint.java b/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
index d5a25fd..46e12c6 100644
--- a/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
@@ -37,6 +37,7 @@ import org.apache.lucene.util.NumericUtils;
  *   <li>{@link #newExactQuery newExactQuery()} for matching an exact 1D point.
  *   <li>{@link #newRangeQuery newRangeQuery()} for matching a 1D range.
  *   <li>{@link #newMultiRangeQuery newMultiRangeQuery()} for matching points/ranges in n-dimensional space.
+ *   <li>{@link #newSetQuery newSetQuery()} for matching a set of 1D values.
  * </ul>
  */
 public final class LongPoint extends Field {
@@ -223,7 +224,7 @@ public final class LongPoint extends Field {
    * Create a query matching any of the specified 1D values.  This is the points equivalent of {@code TermsQuery}.
    * 
    * @param field field name. must not be {@code null}.
-   * @param valuesIn all int values to match
+   * @param valuesIn all values to match
    */
   public static Query newSetQuery(String field, long... valuesIn) throws IOException {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ae709203/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java b/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
index df50c9e..4da1c3e 100644
--- a/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
@@ -42,6 +42,7 @@ import org.apache.lucene.util.StringHelper;
 /** Finds all documents whose point value, previously indexed with e.g. {@link org.apache.lucene.document.LongPoint}, is contained in the
  *  specified set */
 
+// nocommit explain that the 1D case must be pre-sorted
 public class PointInSetQuery extends Query {
   // A little bit overkill for us, since all of our "terms" are always in the same field:
   final PrefixCodedTerms sortedPackedPoints;
@@ -57,7 +58,7 @@ public class PointInSetQuery extends Query {
       throw new IllegalArgumentException("bytesPerDim must be > 0 and <= " + PointValues.MAX_NUM_BYTES + "; got " + bytesPerDim);
     }
     this.bytesPerDim = bytesPerDim;
-    if (numDims < 1 || bytesPerDim > PointValues.MAX_DIMENSIONS) {
+    if (numDims < 1 || numDims > PointValues.MAX_DIMENSIONS) {
       throw new IllegalArgumentException("numDims must be > 0 and <= " + PointValues.MAX_DIMENSIONS + "; got " + numDims);
     }
     this.numDims = numDims;
@@ -73,6 +74,7 @@ public class PointInSetQuery extends Query {
       }
       if (previous == null) {
         previous = new BytesRefBuilder();
+      // nocommit detect out-of-order 1D case
       } else if (previous.get().equals(current)) {
         continue; // deduplicate
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ae709203/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java b/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java
index 4863a88..ee7ed3b 100644
--- a/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java
+++ b/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java
@@ -158,27 +158,18 @@ public final class NumericUtils {
   public static void intToBytes(int x, byte[] dest, int offset) {
     // Flip the sign bit, so negative ints sort before positive ints correctly:
     x ^= 0x80000000;
-    intToBytesDirect(x, dest, offset);
-  }
-
-  public static void intToBytesDirect(int x, byte[] dest, int offset) {
     for (int i = 0; i < 4; i++) {
       dest[offset+i] = (byte) (x >> 24-i*8);
     }
   }
 
-  public static int bytesToInt(byte[] src, int index) {
-    int x = bytesToIntDirect(src, index);
-    // Re-flip the sign bit to restore the original value:
-    return x ^ 0x80000000;
-  }
-
-  public static int bytesToIntDirect(byte[] src, int offset) {
+  public static int bytesToInt(byte[] src, int offset) {
     int x = 0;
     for (int i = 0; i < 4; i++) {
       x |= (src[offset+i] & 0xff) << (24-i*8);
     }
-    return x;
+    // Re-flip the sign bit to restore the original value:
+    return x ^ 0x80000000;
   }
 
   public static void longToBytes(long v, byte[] bytes, int offset) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ae709203/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 5c8306b..f9e6314 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
@@ -28,6 +28,7 @@ import java.util.Set;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.FilterCodec;
 import org.apache.lucene.codecs.PointFormat;
@@ -95,6 +96,60 @@ public class TestPointQueries extends LuceneTestCase {
     }
   }
 
+  public void testBasicFloats() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(new MockAnalyzer(random())));
+
+    Document doc = new Document();
+    doc.add(new FloatPoint("point", -7.0f));
+    w.addDocument(doc);
+
+    doc = new Document();
+    doc.add(new FloatPoint("point", 0.0f));
+    w.addDocument(doc);
+
+    doc = new Document();
+    doc.add(new FloatPoint("point", 3.0f));
+    w.addDocument(doc);
+
+    DirectoryReader r = DirectoryReader.open(w);
+    IndexSearcher s = new IndexSearcher(r);
+    assertEquals(2, s.count(FloatPoint.newRangeQuery("point", -8.0f, false, 1.0f, false)));
+    assertEquals(3, s.count(FloatPoint.newRangeQuery("point", -7.0f, true, 3.0f, true)));
+    assertEquals(1, s.count(FloatPoint.newExactQuery("point", -7.0f)));
+    assertEquals(0, s.count(FloatPoint.newExactQuery("point", -6.0f)));
+    w.close();
+    r.close();
+    dir.close();
+  }
+
+  public void testBasicDoubles() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(new MockAnalyzer(random())));
+
+    Document doc = new Document();
+    doc.add(new DoublePoint("point", -7.0));
+    w.addDocument(doc);
+
+    doc = new Document();
+    doc.add(new DoublePoint("point", 0.0));
+    w.addDocument(doc);
+
+    doc = new Document();
+    doc.add(new DoublePoint("point", 3.0));
+    w.addDocument(doc);
+
+    DirectoryReader r = DirectoryReader.open(w);
+    IndexSearcher s = new IndexSearcher(r);
+    assertEquals(2, s.count(DoublePoint.newRangeQuery("point", -8.0, false, 1.0, false)));
+    assertEquals(3, s.count(DoublePoint.newRangeQuery("point", -7.0, true, 3.0, true)));
+    assertEquals(1, s.count(DoublePoint.newExactQuery("point", -7.0)));
+    assertEquals(0, s.count(DoublePoint.newExactQuery("point", -6.0)));
+    w.close();
+    r.close();
+    dir.close();
+  }
+
   public void testAllEqual() throws Exception {
     int numValues = atLeast(10000);
     long value = randomValue(false);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ae709203/lucene/sandbox/src/java/org/apache/lucene/document/BigIntegerPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/BigIntegerPoint.java b/lucene/sandbox/src/java/org/apache/lucene/document/BigIntegerPoint.java
index 4a1a684..346acb2 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/BigIntegerPoint.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/BigIntegerPoint.java
@@ -16,11 +16,15 @@
  */
 package org.apache.lucene.document;
 
+import java.io.IOException;
 import java.math.BigInteger;
+import java.util.Arrays;
 
+import org.apache.lucene.search.PointInSetQuery;
 import org.apache.lucene.search.PointRangeQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefIterator;
 import org.apache.lucene.util.NumericUtils;
 
 /** 
@@ -34,6 +38,7 @@ import org.apache.lucene.util.NumericUtils;
  *   <li>{@link #newExactQuery newExactQuery()} for matching an exact 1D point.
  *   <li>{@link #newRangeQuery newRangeQuery()} for matching a 1D range.
  *   <li>{@link #newMultiRangeQuery newMultiRangeQuery()} for matching points/ranges in n-dimensional space.
+ *   <li>{@link #newSetQuery newSetQuery()} for matching a set of 1D values.
  * </ul>
  */
 public class BigIntegerPoint extends Field {
@@ -214,5 +219,42 @@ public class BigIntegerPoint extends Field {
     };
   }
 
-  // nocommit newSetQuery
+  /**
+   * Create a query matching any of the specified 1D values.  This is the points equivalent of {@code TermsQuery}.
+   * 
+   * @param field field name. must not be {@code null}.
+   * @param valuesIn all values to match
+   */
+  public static Query newSetQuery(String field, BigInteger... valuesIn) throws IOException {
+
+    // Don't unexpectedly change the user's incoming values array:
+    BigInteger[] values = valuesIn.clone();
+
+    Arrays.sort(values);
+
+    final BytesRef value = new BytesRef(new byte[BYTES]);
+
+    return new PointInSetQuery(field, 1, BYTES,
+                               new BytesRefIterator() {
+
+                                 int upto;
+
+                                 @Override
+                                 public BytesRef next() {
+                                   if (upto == values.length) {
+                                     return null;
+                                   } else {
+                                     encodeDimension(values[upto], value.bytes, 0);
+                                     upto++;
+                                     return value;
+                                   }
+                                 }
+                               }) {
+      @Override
+      protected String toString(byte[] value) {
+        assert value.length == BYTES;
+        return decodeDimension(value, 0).toString();
+      }
+    };
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ae709203/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressPoint.java b/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressPoint.java
index 090842e..3081875 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressPoint.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressPoint.java
@@ -16,12 +16,16 @@
  */
 package org.apache.lucene.document;
 
+import java.io.IOException;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
+import java.util.Arrays;
 
+import org.apache.lucene.search.PointInSetQuery;
 import org.apache.lucene.search.PointRangeQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefIterator;
 
 /** 
  * A field indexing {@link InetAddress} dimensionally such that finding
@@ -34,7 +38,7 @@ import org.apache.lucene.util.BytesRef;
  *   <li>{@link #newExactQuery newExactQuery()} for matching an exact network address.
  *   <li>{@link #newPrefixQuery newPrefixQuery()} for matching a network based on CIDR prefix.
  *   <li>{@link #newRangeQuery newRangeQuery()} for matching arbitrary network address ranges.
- *   <li>{@link #newSetQuery newSetQuery()} for matching a set of addresses.
+ *   <li>{@link #newSetQuery newSetQuery()} for matching a set of 1D values.
  * </ul>
  * <p>
  * This field supports both IPv4 and IPv6 addresses: IPv4 addresses are converted
@@ -212,7 +216,7 @@ public class InetAddressPoint extends Field {
    * Create a query matching any of the specified 1D values.  This is the points equivalent of {@code TermsQuery}.
    * 
    * @param field field name. must not be {@code null}.
-   * @param valuesIn all int values to match
+   * @param valuesIn all values to match
    */
   public static Query newSetQuery(String field, InetAddress... valuesIn) throws IOException {
 
@@ -233,7 +237,8 @@ public class InetAddressPoint extends Field {
                                    if (upto == values.length) {
                                      return null;
                                    } else {
-                                     encode(values[upto], value.bytes, 0);
+                                     value.bytes = encode(values[upto]);
+                                     assert value.bytes.length == value.length;
                                      upto++;
                                      return value;
                                    }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ae709203/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 bdd42b1..ff58ae0 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java
@@ -129,8 +129,6 @@ public class LatLonPoint extends Field {
     return x / LON_SCALE;
   }
 
-  // nocommit newSetQuery
-  
   /** Turns quantized value from byte array back into a double. */
   public static double decodeLon(byte[] src, int offset) {
     return decodeLon(NumericUtils.bytesToInt(src, offset));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ae709203/lucene/sandbox/src/test/org/apache/lucene/document/TestBigIntegerPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestBigIntegerPoint.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestBigIntegerPoint.java
index d97f6fd..6b0d696 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestBigIntegerPoint.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestBigIntegerPoint.java
@@ -43,6 +43,8 @@ public class TestBigIntegerPoint extends LuceneTestCase {
     IndexSearcher searcher = newSearcher(reader);
     assertEquals(1, searcher.count(BigIntegerPoint.newExactQuery("field", large)));
     assertEquals(1, searcher.count(BigIntegerPoint.newRangeQuery("field", large.subtract(BigInteger.ONE), false, large.add(BigInteger.ONE), false)));
+    assertEquals(1, searcher.count(BigIntegerPoint.newSetQuery("field", large)));
+    assertEquals(0, searcher.count(BigIntegerPoint.newSetQuery("field", large.subtract(BigInteger.ONE))));
 
     reader.close();
     writer.close();
@@ -83,5 +85,13 @@ public class TestBigIntegerPoint extends LuceneTestCase {
   public void testToString() throws Exception {
     assertEquals("BigIntegerPoint <field:1>", new BigIntegerPoint("field", BigInteger.ONE).toString());
     assertEquals("BigIntegerPoint <field:1,-2>", new BigIntegerPoint("field", BigInteger.ONE, BigInteger.valueOf(-2)).toString());
+    assertEquals("field:[1 TO 1]", BigIntegerPoint.newExactQuery("field", BigInteger.ONE).toString());
+    assertEquals("field:{1 TO 17]", BigIntegerPoint.newRangeQuery("field", BigInteger.ONE, false, BigInteger.valueOf(17), true).toString());
+    assertEquals("field:{1 TO 17],[0 TO 42}", BigIntegerPoint.newMultiRangeQuery("field",
+                                                                                 new BigInteger[] {BigInteger.ONE, BigInteger.ZERO},
+                                                                                 new boolean[] {false, true},
+                                                                                 new BigInteger[] {BigInteger.valueOf(17), BigInteger.valueOf(42)},
+                                                                                 new boolean[] {true, false}).toString());
+    assertEquals("field:{1}", BigIntegerPoint.newSetQuery("field", BigInteger.ONE).toString());
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ae709203/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
index bdb3861..500edef 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
@@ -44,8 +44,8 @@ public class TestInetAddressPoint extends LuceneTestCase {
     assertEquals(1, searcher.count(InetAddressPoint.newExactQuery("field", address)));
     assertEquals(1, searcher.count(InetAddressPoint.newPrefixQuery("field", address, 24)));
     assertEquals(1, searcher.count(InetAddressPoint.newRangeQuery("field", InetAddress.getByName("1.2.3.3"), false, InetAddress.getByName("1.2.3.5"), false)));
-    assertEquals(1, searcher.count(InetAddressPoint.newSetQuery("field", InetAddress.getByName("1.2.3.3")));
-    assertEquals(0, searcher.count(InetAddressPoint.newSetQuery("field", InetAddress.getByName("1.2.3.4")));
+    assertEquals(1, searcher.count(InetAddressPoint.newSetQuery("field", InetAddress.getByName("1.2.3.4"))));
+    assertEquals(0, searcher.count(InetAddressPoint.newSetQuery("field", InetAddress.getByName("1.2.3.3"))));
 
     reader.close();
     writer.close();
@@ -85,5 +85,6 @@ public class TestInetAddressPoint extends LuceneTestCase {
     
     assertEquals("field:[1.2.3.0 TO 1.2.3.255]", InetAddressPoint.newPrefixQuery("field", InetAddress.getByName("1.2.3.4"), 24).toString());
     assertEquals("field:[fdc8:57ed:f042:ad1:0:0:0:0 TO fdc8:57ed:f042:ad1:ffff:ffff:ffff:ffff]", InetAddressPoint.newPrefixQuery("field", InetAddress.getByName("fdc8:57ed:f042:0ad1:f66d:4ff:fe90:ce0c"), 64).toString());
+    assertEquals("field:{fdc8:57ed:f042:ad1:f66d:4ff:fe90:ce0c}", InetAddressPoint.newSetQuery("field", InetAddress.getByName("fdc8:57ed:f042:0ad1:f66d:4ff:fe90:ce0c")).toString());
   }
 }


[02/18] lucene-solr git commit: factor out some private helper classes

Posted by mi...@apache.org.
factor out some private helper classes


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

Branch: refs/heads/master
Commit: 96ed42bb135d22494bfe76ec5b85ca580ee8fb84
Parents: 013abea
Author: Mike McCandless <mi...@apache.org>
Authored: Tue Feb 23 16:48:35 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Tue Feb 23 16:48:35 2016 -0500

----------------------------------------------------------------------
 .../apache/lucene/search/PointInSetQuery.java   | 304 ++++++++++---------
 1 file changed, 167 insertions(+), 137 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/96ed42bb/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java b/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
index 4403e45..bbc9a54 100644
--- a/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
@@ -64,6 +64,7 @@ public class PointInSetQuery extends Query {
     BytesRefBuilder previous = null;
     BytesRef current;
     while ((current = packedPoints.next()) != null) {
+      // nocommit make sure a test tests this:
       if (current.length != numDims * bytesPerDim) {
         throw new IllegalArgumentException("packed point length should be " + (numDims * bytesPerDim) + " but got " + current.length + "; field=\"" + field + "\", numDims=" + numDims + " bytesPerDim=" + bytesPerDim);
       }
@@ -145,148 +146,21 @@ public class PointInSetQuery extends Query {
         DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc());
 
         int[] hitCount = new int[1];
-        final TermIterator iterator = sortedPackedPoints.iterator();
-        byte[] pointBytes = new byte[bytesPerDim * numDims];
 
         if (numDims == 1) {
 
-          final BytesRef scratch = new BytesRef();
-          scratch.length = bytesPerDim;
-
-          // Optimize this common case, effectively doing a merge sort of the indexed values vs the queried set:
-          values.intersect(field,
-                           new IntersectVisitor() {
-
-                             private BytesRef nextQueryPoint = iterator.next();
-
-                             @Override
-                             public void grow(int count) {
-                               result.grow(count);
-                             }
-
-                             @Override
-                             public void visit(int docID) {
-                               hitCount[0]++;
-                               result.add(docID);
-                             }
-
-                             @Override
-                             public void visit(int docID, byte[] packedValue) {
-                               scratch.bytes = packedValue;
-                               while (nextQueryPoint != null) {
-                                 int cmp = nextQueryPoint.compareTo(scratch);
-                                 if (cmp == 0) {
-                                   // Query point equals index point, so collect and return
-                                   hitCount[0]++;
-                                   result.add(docID);
-                                   break;
-                                 } else if (cmp < 0) {
-                                   // Query point is before index point, so we move to next query point
-                                   nextQueryPoint = iterator.next();
-                                 } else {
-                                   // Query point is after index point, so we don't collect and we return:
-                                   break;
-                                 }
-                               }
-                             }
-
-                             @Override
-                             public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
-
-                               while (nextQueryPoint != null) {
-                                 scratch.bytes = minPackedValue;
-                                 int cmpMin = nextQueryPoint.compareTo(scratch);
-                                 if (cmpMin < 0) {
-                                   // query point is before the start of this cell
-                                   nextQueryPoint = iterator.next();
-                                   continue;
-                                 }
-                                 scratch.bytes = maxPackedValue;
-                                 int cmpMax = nextQueryPoint.compareTo(scratch);
-                                 if (cmpMax > 0) {
-                                   // query point is after the end of this cell
-                                   return Relation.CELL_OUTSIDE_QUERY;
-                                 }
-
-                                 if (cmpMin == 0 && cmpMax == 0) {
-                                   // NOTE: we only hit this if we are on a cell whose min and max values are exactly equal to our point,
-                                   // which can easily happen if many (> 1024) docs share this one value
-                                   return Relation.CELL_INSIDE_QUERY;
-                                 } else {
-                                   return Relation.CELL_CROSSES_QUERY;
-                                 }
-                               }
-
-                               // We exhausted all points in the query:
-                               return Relation.CELL_OUTSIDE_QUERY;
-                             }
-                           });
+          // We optimize this common case, effectively doing a merge sort of the indexed values vs the queried set:
+          values.intersect(field, new MergePointVisitor(sortedPackedPoints.iterator(), hitCount, result));
+
         } else {
+          // NOTE: this is naive implementation, where for each point we re-walk the KD tree to intersect.  We could instead do a similar
+          // optimization as the 1D case, but I think it'd mean building a query-time KD tree so we could efficiently intersect against the
+          // index, which is probably tricky!
+          SinglePointVisitor visitor = new SinglePointVisitor(hitCount, result);
+          TermIterator iterator = sortedPackedPoints.iterator();
           for (BytesRef point = iterator.next(); point != null; point = iterator.next()) {
-            // nocommit make sure a test tests this:
-            assert point.length == pointBytes.length;
-            System.arraycopy(point.bytes, point.offset, pointBytes, 0, pointBytes.length);
-
-            final BytesRef finalPoint = point;
-
-            values.intersect(field,
-                             // nocommit don't make new instance of this for each point?
-                             new IntersectVisitor() {
-
-                               @Override
-                               public void grow(int count) {
-                                 result.grow(count);
-                               }
-
-                               @Override
-                               public void visit(int docID) {
-                                 hitCount[0]++;
-                                 result.add(docID);
-                               }
-
-                               @Override
-                               public void visit(int docID, byte[] packedValue) {
-                                 assert packedValue.length == finalPoint.length;
-                                 if (Arrays.equals(packedValue, pointBytes)) {
-                                   // The point for this doc matches the point we are querying on
-                                   hitCount[0]++;
-                                   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;
-
-                                   int cmpMin = StringHelper.compare(bytesPerDim, minPackedValue, offset, pointBytes, offset);
-                                   if (cmpMin > 0) {
-                                     return Relation.CELL_OUTSIDE_QUERY;
-                                   }
-
-                                   int cmpMax = StringHelper.compare(bytesPerDim, maxPackedValue, offset, pointBytes, offset);
-                                   if (cmpMax < 0) {
-                                     return Relation.CELL_OUTSIDE_QUERY;
-                                   }
-
-                                   if (cmpMin != 0 || cmpMax != 0) {
-                                     crosses = true;
-                                   }
-                                 }
-
-                                 if (crosses) {
-                                   return Relation.CELL_CROSSES_QUERY;
-                                 } else {
-                                   // nocommit make sure tests hit this case:
-                                   // NOTE: we only hit this if we are on a cell whose min and max values are exactly equal to our point,
-                                   // which can easily happen if many docs share this one value
-                                   return Relation.CELL_INSIDE_QUERY;
-                                 }
-                               }
-                             });
+            visitor.setPoint(point);
+            values.intersect(field, visitor);
           }
         }
 
@@ -296,6 +170,162 @@ public class PointInSetQuery extends Query {
     };
   }
 
+  /** Essentially does a merge sort, only collecting hits when the indexed point and query point are the same.  This is an optimization,
+   *  used in the 1D case. */
+  private class MergePointVisitor implements IntersectVisitor {
+
+    private final DocIdSetBuilder result;
+    private final int[] hitCount;
+    private final TermIterator iterator;
+    private BytesRef nextQueryPoint;
+    private final BytesRef scratch = new BytesRef();
+
+    public MergePointVisitor(TermIterator iterator, int[] hitCount, DocIdSetBuilder result) throws IOException {
+      this.hitCount = hitCount;
+      this.result = result;
+      this.iterator = iterator;
+      nextQueryPoint = iterator.next();
+      scratch.length = bytesPerDim;
+    }
+
+    @Override
+    public void grow(int count) {
+      result.grow(count);
+    }
+
+    @Override
+    public void visit(int docID) {
+      hitCount[0]++;
+      result.add(docID);
+    }
+
+    @Override
+    public void visit(int docID, byte[] packedValue) {
+      scratch.bytes = packedValue;
+      while (nextQueryPoint != null) {
+        int cmp = nextQueryPoint.compareTo(scratch);
+        if (cmp == 0) {
+          // Query point equals index point, so collect and return
+          hitCount[0]++;
+          result.add(docID);
+          break;
+        } else if (cmp < 0) {
+          // Query point is before index point, so we move to next query point
+          nextQueryPoint = iterator.next();
+        } else {
+          // Query point is after index point, so we don't collect and we return:
+          break;
+        }
+      }
+    }
+
+    @Override
+    public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+
+      while (nextQueryPoint != null) {
+        scratch.bytes = minPackedValue;
+        int cmpMin = nextQueryPoint.compareTo(scratch);
+        if (cmpMin < 0) {
+          // query point is before the start of this cell
+          nextQueryPoint = iterator.next();
+          continue;
+        }
+        scratch.bytes = maxPackedValue;
+        int cmpMax = nextQueryPoint.compareTo(scratch);
+        if (cmpMax > 0) {
+          // query point is after the end of this cell
+          return Relation.CELL_OUTSIDE_QUERY;
+        }
+
+        if (cmpMin == 0 && cmpMax == 0) {
+          // NOTE: we only hit this if we are on a cell whose min and max values are exactly equal to our point,
+          // which can easily happen if many (> 1024) docs share this one value
+          return Relation.CELL_INSIDE_QUERY;
+        } else {
+          return Relation.CELL_CROSSES_QUERY;
+        }
+      }
+
+      // We exhausted all points in the query:
+      return Relation.CELL_OUTSIDE_QUERY;
+    }
+  }
+
+  /** IntersectVisitor that queries against a highly degenerate shape: a single point.  This is used in the > 1D case. */
+  private class SinglePointVisitor implements IntersectVisitor {
+
+    private final DocIdSetBuilder result;
+    private final int[] hitCount;
+    public BytesRef point;
+    private final byte[] pointBytes;
+
+    public SinglePointVisitor(int[] hitCount, DocIdSetBuilder result) {
+      this.hitCount = hitCount;
+      this.result = result;
+      this.pointBytes = new byte[bytesPerDim * numDims];
+    }
+
+    public void setPoint(BytesRef point) {
+      // we verified this up front in query's ctor:
+      assert point.length == pointBytes.length;
+      System.arraycopy(point.bytes, point.offset, pointBytes, 0, pointBytes.length);
+    }
+
+    @Override
+    public void grow(int count) {
+      result.grow(count);
+    }
+
+    @Override
+    public void visit(int docID) {
+      hitCount[0]++;
+      result.add(docID);
+    }
+
+    @Override
+    public void visit(int docID, byte[] packedValue) {
+      assert packedValue.length == point.length;
+      if (Arrays.equals(packedValue, pointBytes)) {
+        // The point for this doc matches the point we are querying on
+        hitCount[0]++;
+        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;
+
+        int cmpMin = StringHelper.compare(bytesPerDim, minPackedValue, offset, pointBytes, offset);
+        if (cmpMin > 0) {
+          return Relation.CELL_OUTSIDE_QUERY;
+        }
+
+        int cmpMax = StringHelper.compare(bytesPerDim, maxPackedValue, offset, pointBytes, offset);
+        if (cmpMax < 0) {
+          return Relation.CELL_OUTSIDE_QUERY;
+        }
+
+        if (cmpMin != 0 || cmpMax != 0) {
+          crosses = true;
+        }
+      }
+
+      if (crosses) {
+        return Relation.CELL_CROSSES_QUERY;
+      } else {
+        // nocommit make sure tests hit this case:
+        // NOTE: we only hit this if we are on a cell whose min and max values are exactly equal to our point,
+        // which can easily happen if many docs share this one value
+        return Relation.CELL_INSIDE_QUERY;
+      }
+    }
+  }
+
   @Override
   public int hashCode() {
     int hash = super.hashCode();


[16/18] lucene-solr git commit: fix compilation errors

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

Branch: refs/heads/master
Commit: 7e1fbdf31a8b835c3c462e849efa30e816cfb3c8
Parents: a1f5269
Author: Mike McCandless <mi...@apache.org>
Authored: Thu Feb 25 05:51:18 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Thu Feb 25 05:51:18 2016 -0500

----------------------------------------------------------------------
 .../src/test/org/apache/lucene/document/TestBigIntegerPoint.java   | 2 +-
 .../src/test/org/apache/lucene/document/TestInetAddressPoint.java  | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e1fbdf3/lucene/sandbox/src/test/org/apache/lucene/document/TestBigIntegerPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestBigIntegerPoint.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestBigIntegerPoint.java
index b5d8022..3e8cf3d 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestBigIntegerPoint.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestBigIntegerPoint.java
@@ -45,7 +45,7 @@ public class TestBigIntegerPoint extends LuceneTestCase {
     assertEquals(1, searcher.count(BigIntegerPoint.newRangeQuery("field", large.subtract(BigInteger.ONE), false, large.add(BigInteger.ONE), false)));
     assertEquals(1, searcher.count(BigIntegerPoint.newSetQuery("field", large)));
     assertEquals(0, searcher.count(BigIntegerPoint.newSetQuery("field", large.subtract(BigInteger.ONE))));
-    assertEquals(0, searcher.count(BigIntegerPoint.newSetQuery("field"));
+    assertEquals(0, searcher.count(BigIntegerPoint.newSetQuery("field")));
 
     reader.close();
     writer.close();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7e1fbdf3/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
index b2f41c4..c9be31f 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
@@ -46,7 +46,7 @@ public class TestInetAddressPoint extends LuceneTestCase {
     assertEquals(1, searcher.count(InetAddressPoint.newRangeQuery("field", InetAddress.getByName("1.2.3.3"), false, InetAddress.getByName("1.2.3.5"), false)));
     assertEquals(1, searcher.count(InetAddressPoint.newSetQuery("field", InetAddress.getByName("1.2.3.4"))));
     assertEquals(0, searcher.count(InetAddressPoint.newSetQuery("field", InetAddress.getByName("1.2.3.3"))));
-    assertEquals(0, searcher.count(InetAddressPoint.newSetQuery("field"));
+    assertEquals(0, searcher.count(InetAddressPoint.newSetQuery("field")));
 
     reader.close();
     writer.close();