You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by nk...@apache.org on 2017/02/16 18:46:14 UTC

lucene-solr:master: LUCENE-7449: Add CROSSES relation support to RangeFieldQuery.

Repository: lucene-solr
Updated Branches:
  refs/heads/master d49edabf8 -> 6cbda0266


LUCENE-7449: Add CROSSES relation support to RangeFieldQuery.


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

Branch: refs/heads/master
Commit: 6cbda026633ccd07c07e6db12561c0110a9eec4c
Parents: d49edab
Author: Nicholas Knize <nk...@gmail.com>
Authored: Thu Feb 16 12:42:52 2017 -0600
Committer: Nicholas Knize <nk...@gmail.com>
Committed: Thu Feb 16 12:46:03 2017 -0600

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  2 +
 .../lucene/document/DoubleRangeField.java       | 20 +++++
 .../apache/lucene/document/FloatRangeField.java | 20 +++++
 .../apache/lucene/document/IntRangeField.java   | 20 +++++
 .../apache/lucene/document/LongRangeField.java  | 20 +++++
 .../apache/lucene/document/RangeFieldQuery.java | 88 +++++++++++++-------
 .../search/BaseRangeFieldQueryTestCase.java     | 18 ++--
 .../search/TestDoubleRangeFieldQueries.java     |  9 +-
 .../search/TestFloatRangeFieldQueries.java      |  9 +-
 .../lucene/search/TestIntRangeFieldQueries.java |  9 +-
 .../search/TestLongRangeFieldQueries.java       |  9 +-
 11 files changed, 183 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6cbda026/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 4a9dc61..f9fdf7a 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -90,6 +90,8 @@ API Changes
 
 New Features
 
+* LUCENE-7449: Add CROSSES relation support to RangeFieldQuery. (Nick Knize)
+
 * LUCENE-7623: Add FunctionScoreQuery and FunctionMatchQuery (Alan Woodward,
   Adrien Grand, David Smiley)
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6cbda026/lucene/sandbox/src/java/org/apache/lucene/document/DoubleRangeField.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/DoubleRangeField.java b/lucene/sandbox/src/java/org/apache/lucene/document/DoubleRangeField.java
index 2af8697..c5ae0e7 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/DoubleRangeField.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/DoubleRangeField.java
@@ -231,6 +231,26 @@ public class DoubleRangeField extends Field {
     };
   }
 
+  /**
+   * Create a query for matching indexed ranges that cross the defined range.
+   * A CROSSES is defined as any set of ranges that are not disjoint and not wholly contained by
+   * the query. Effectively, its the complement of union(WITHIN, DISJOINT).
+   * @param field field name. must not be null.
+   * @param min array of min values. (accepts {@code Double.MIN_VALUE})
+   * @param max array of max values. (accepts {@code Double.MAX_VALUE})
+   * @return query for matching ranges within the defined range
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newCrossesQuery(String field, final double[] min, final double[] max) {
+    checkArgs(min, max);
+    return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.CROSSES) {
+      @Override
+      protected String toString(byte[] ranges, int dimension) {
+        return DoubleRangeField.toString(ranges, dimension);
+      }
+    };
+  }
+
   @Override
   public String toString() {
     StringBuilder sb = new StringBuilder();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6cbda026/lucene/sandbox/src/java/org/apache/lucene/document/FloatRangeField.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/FloatRangeField.java b/lucene/sandbox/src/java/org/apache/lucene/document/FloatRangeField.java
index e138ae2..60e0acf 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/FloatRangeField.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/FloatRangeField.java
@@ -231,6 +231,26 @@ public class FloatRangeField extends Field {
     };
   }
 
+  /**
+   * Create a query for matching indexed ranges that cross the defined range.
+   * A CROSSES is defined as any set of ranges that are not disjoint and not wholly contained by
+   * the query. Effectively, its the complement of union(WITHIN, DISJOINT).
+   * @param field field name. must not be null.
+   * @param min array of min values. (accepts {@code Float.NEGATIVE_INFINITY})
+   * @param max array of max values. (accepts {@code Float.POSITIVE_INFINITY})
+   * @return query for matching ranges within the defined range
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newCrossesQuery(String field, final float[] min, final float[] max) {
+    checkArgs(min, max);
+    return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.CROSSES) {
+      @Override
+      protected String toString(byte[] ranges, int dimension) {
+        return FloatRangeField.toString(ranges, dimension);
+      }
+    };
+  }
+
   @Override
   public String toString() {
     StringBuilder sb = new StringBuilder();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6cbda026/lucene/sandbox/src/java/org/apache/lucene/document/IntRangeField.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/IntRangeField.java b/lucene/sandbox/src/java/org/apache/lucene/document/IntRangeField.java
index c0ce61d..53a3311 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/IntRangeField.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/IntRangeField.java
@@ -231,6 +231,26 @@ public class IntRangeField extends Field {
     };
   }
 
+  /**
+   * Create a query for matching indexed ranges that cross the defined range.
+   * A CROSSES is defined as any set of ranges that are not disjoint and not wholly contained by
+   * the query. Effectively, its the complement of union(WITHIN, DISJOINT).
+   * @param field field name. must not be null.
+   * @param min array of min values. (accepts {@code Integer.MIN_VALUE})
+   * @param max array of max values. (accepts {@code Integer.MAX_VALUE})
+   * @return query for matching ranges within the defined range
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newCrossesQuery(String field, final int[] min, final int[] max) {
+    checkArgs(min, max);
+    return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.CROSSES) {
+      @Override
+      protected String toString(byte[] ranges, int dimension) {
+        return IntRangeField.toString(ranges, dimension);
+      }
+    };
+  }
+
   @Override
   public String toString() {
     StringBuilder sb = new StringBuilder();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6cbda026/lucene/sandbox/src/java/org/apache/lucene/document/LongRangeField.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LongRangeField.java b/lucene/sandbox/src/java/org/apache/lucene/document/LongRangeField.java
index b9298b9..7addaf5 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LongRangeField.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LongRangeField.java
@@ -229,6 +229,26 @@ public class LongRangeField extends Field {
     };
   }
 
+  /**
+   * Create a query for matching indexed ranges that cross the defined range.
+   * A CROSSES is defined as any set of ranges that are not disjoint and not wholly contained by
+   * the query. Effectively, its the complement of union(WITHIN, DISJOINT).
+   * @param field field name. must not be null.
+   * @param min array of min values. (accepts {@code Long.MIN_VALUE})
+   * @param max array of max values. (accepts {@code Long.MAX_VALUE})
+   * @return query for matching ranges within the defined range
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newCrossesQuery(String field, final long[] min, final long[] max) {
+    checkArgs(min, max);
+    return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.CROSSES) {
+      @Override
+      protected String toString(byte[] ranges, int dimension) {
+        return LongRangeField.toString(ranges, dimension);
+      }
+    };
+  }
+
   @Override
   public String toString() {
     StringBuilder sb = new StringBuilder();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6cbda026/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java b/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java
index 5249191..da7f3b6 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java
@@ -19,6 +19,8 @@ package org.apache.lucene.document;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Objects;
+import java.util.function.IntPredicate;
+import java.util.function.Predicate;
 
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.LeafReader;
@@ -62,7 +64,9 @@ abstract class RangeFieldQuery extends Query {
     /** Use this for within queries. */
     WITHIN,
     /** Use this for contains */
-    CONTAINS
+    CONTAINS,
+    /** Use this for crosses queries */
+    CROSSES
   }
 
   /**
@@ -107,7 +111,7 @@ abstract class RangeFieldQuery extends Query {
   @Override
   public final Weight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException {
     return new ConstantScoreWeight(this, boost) {
-      final RangeFieldComparator comparator = new RangeFieldComparator();
+      final RangeFieldComparator target = new RangeFieldComparator();
       private DocIdSet buildMatchingDocIdSet(LeafReader reader, PointValues values) throws IOException {
         DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc(), values, field);
         values.intersect(
@@ -123,14 +127,7 @@ abstract class RangeFieldQuery extends Query {
               }
               @Override
               public void visit(int docID, byte[] leaf) throws IOException {
-                // add the document iff:
-                if (Arrays.equals(ranges, leaf)
-                    // target is within cell and queryType is INTERSECTS or CONTAINS:
-                    || (comparator.isWithin(leaf) && queryType != QueryType.WITHIN)
-                    // target contains cell and queryType is INTERSECTS or WITHIN:
-                    || (comparator.contains(leaf) && queryType != QueryType.CONTAINS)
-                    // target is not disjoint (crosses) and queryType is INTERSECTS
-                    || (comparator.isDisjoint(leaf) == false && queryType == QueryType.INTERSECTS)) {
+                if (target.matches(leaf)) {
                   adder.add(docID);
                 }
               }
@@ -138,14 +135,15 @@ abstract class RangeFieldQuery extends Query {
               public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
                 byte[] node = getInternalRange(minPackedValue, maxPackedValue);
                 // compute range relation for BKD traversal
-                if (comparator.isDisjoint(node)) {
+                if (target.intersects(node) == false) {
                   return Relation.CELL_OUTSIDE_QUERY;
-                } else if (comparator.isWithin(node)) {
+                } else if (target.within(node)) {
                   // target within cell; continue traversing:
                   return Relation.CELL_CROSSES_QUERY;
-                } else if (comparator.contains(node)) {
-                  // target contains cell; add iff queryType is not a CONTAINS query:
-                  return (queryType == QueryType.CONTAINS) ? Relation.CELL_OUTSIDE_QUERY : Relation.CELL_INSIDE_QUERY;
+                } else if (target.contains(node)) {
+                  // target contains cell; add iff queryType is not a CONTAINS or CROSSES query:
+                  return (queryType == QueryType.CONTAINS || queryType == QueryType.CROSSES) ?
+                      Relation.CELL_OUTSIDE_QUERY : Relation.CELL_INSIDE_QUERY;
                 }
                 // target intersects cell; continue traversing:
                 return Relation.CELL_CROSSES_QUERY;
@@ -174,7 +172,7 @@ abstract class RangeFieldQuery extends Query {
           byte[] range = getInternalRange(values.getMinPackedValue(), values.getMaxPackedValue());
           // if the internal node is not equal and not contained by the query, all docs do not match
           if ((!Arrays.equals(ranges, range)
-              && (comparator.contains(range) && queryType != QueryType.CONTAINS)) == false) {
+              && (target.contains(range) && queryType != QueryType.CONTAINS)) == false) {
             allDocsMatch = false;
           }
         } else {
@@ -203,30 +201,56 @@ abstract class RangeFieldQuery extends Query {
    * {@code RangeField} types based on the defined query range and relation.
    */
   class RangeFieldComparator {
-    /** check if the query is outside the candidate range */
-    private boolean isDisjoint(final byte[] range) {
-      for (int d=0; d<numDims; ++d) {
-        if (compareMinMax(range, d) > 0 || compareMaxMin(range, d) < 0) {
-          return true;
-        }
+    final Predicate<byte[]> predicate;
+
+    /** constructs the comparator based on the query type */
+    RangeFieldComparator() {
+      switch (queryType) {
+        case INTERSECTS:
+          predicate = this::intersects;
+          break;
+        case WITHIN:
+          predicate = this::contains;
+          break;
+        case CONTAINS:
+          predicate = this::within;
+          break;
+        case CROSSES:
+          // crosses first checks intersection (disjoint automatic fails),
+          // then ensures the query doesn't wholly contain the leaf:
+          predicate = (byte[] leaf) -> this.intersects(leaf)
+              && this.contains(leaf) == false;
+          break;
+        default:
+          throw new IllegalArgumentException("invalid queryType [" + queryType + "] found.");
       }
-      return false;
+    }
+
+    /** determines if the candidate range matches the query request */
+    private boolean matches(final byte[] candidate) {
+      return (Arrays.equals(ranges, candidate) && queryType != QueryType.CROSSES)
+          || predicate.test(candidate);
+    }
+
+    /** check if query intersects candidate range */
+    private boolean intersects(final byte[] candidate) {
+      return relate((int d) -> compareMinMax(candidate, d) > 0 || compareMaxMin(candidate, d) < 0);
     }
 
     /** check if query is within candidate range */
-    private boolean isWithin(final byte[] range) {
-      for (int d=0; d<numDims; ++d) {
-        if (compareMinMin(range, d) < 0 || compareMaxMax(range, d) > 0) {
-          return false;
-        }
-      }
-      return true;
+    private boolean within(final byte[] candidate) {
+      return relate((int d) -> compareMinMin(candidate, d) < 0 || compareMaxMax(candidate, d) > 0);
     }
 
     /** check if query contains candidate range */
-    private boolean contains(final byte[] range) {
+    private boolean contains(final byte[] candidate) {
+      return relate((int d) -> compareMinMin(candidate, d) > 0 || compareMaxMax(candidate, d) < 0);
+    }
+
+    /** internal method used by each relation method to test range relation logic */
+    private boolean relate(IntPredicate predicate) {
       for (int d=0; d<numDims; ++d) {
-        if (compareMinMin(range, d) > 0 || compareMaxMax(range, d) < 0) {
+        if (predicate.test(d)) {
           return false;
         }
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6cbda026/lucene/sandbox/src/test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java b/lucene/sandbox/src/test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java
index ceafd53..53f3b82 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java
@@ -51,6 +51,8 @@ public abstract class BaseRangeFieldQueryTestCase extends LuceneTestCase {
 
   protected abstract Query newWithinQuery(Range box);
 
+  protected abstract Query newCrossesQuery(Range box);
+
   protected abstract Range nextRange(int dimensions);
 
   protected int dimension() {
@@ -213,7 +215,7 @@ public abstract class BaseRangeFieldQueryTestCase extends LuceneTestCase {
 
       // occasionally test open ended bounding ranges
       Range queryRange = nextRange(dimensions);
-      int rv = random().nextInt(3);
+      int rv = random().nextInt(4);
       Query query;
       Range.QueryType queryType;
       if (rv == 0) {
@@ -222,9 +224,12 @@ public abstract class BaseRangeFieldQueryTestCase extends LuceneTestCase {
       } else if (rv == 1)  {
         queryType = Range.QueryType.CONTAINS;
         query = newContainsQuery(queryRange);
-      } else {
+      } else if (rv == 2) {
         queryType = Range.QueryType.WITHIN;
         query = newWithinQuery(queryRange);
+      } else {
+        queryType = Range.QueryType.CROSSES;
+        query = newCrossesQuery(queryRange);
       }
 
       if (VERBOSE) {
@@ -296,12 +301,15 @@ public abstract class BaseRangeFieldQueryTestCase extends LuceneTestCase {
   }
 
   protected boolean expectedBBoxQueryResult(Range queryRange, Range range, Range.QueryType queryType) {
-    if (queryRange.isEqual(range)) {
+    if (queryRange.isEqual(range) && queryType != Range.QueryType.CROSSES) {
       return true;
     }
     Range.QueryType relation = range.relate(queryRange);
     if (queryType == Range.QueryType.INTERSECTS) {
       return relation != null;
+    } else if (queryType == Range.QueryType.CROSSES) {
+      // by definition, RangeFields that CONTAIN the query are also considered to cross
+      return relation == queryType || relation == Range.QueryType.CONTAINS;
     }
     return relation == queryType;
   }
@@ -309,7 +317,7 @@ public abstract class BaseRangeFieldQueryTestCase extends LuceneTestCase {
   abstract static class Range {
     protected boolean isMissing = false;
 
-    enum QueryType { INTERSECTS, WITHIN, CONTAINS }
+    enum QueryType { INTERSECTS, WITHIN, CONTAINS, CROSSES }
 
     protected abstract int numDimensions();
     protected abstract Object getMin(int dim);
@@ -330,7 +338,7 @@ public abstract class BaseRangeFieldQueryTestCase extends LuceneTestCase {
       } else if (contains(other)) {
         return QueryType.CONTAINS;
       }
-      return QueryType.INTERSECTS;
+      return QueryType.CROSSES;
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6cbda026/lucene/sandbox/src/test/org/apache/lucene/search/TestDoubleRangeFieldQueries.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/TestDoubleRangeFieldQueries.java b/lucene/sandbox/src/test/org/apache/lucene/search/TestDoubleRangeFieldQueries.java
index ad7e6b1..43630e3 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/search/TestDoubleRangeFieldQueries.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/TestDoubleRangeFieldQueries.java
@@ -73,6 +73,11 @@ public class TestDoubleRangeFieldQueries extends BaseRangeFieldQueryTestCase {
     return DoubleRangeField.newWithinQuery(FIELD_NAME, ((DoubleRange)r).min, ((DoubleRange)r).max);
   }
 
+  @Override
+  protected Query newCrossesQuery(Range r) {
+    return DoubleRangeField.newCrossesQuery(FIELD_NAME, ((DoubleRange)r).min, ((DoubleRange)r).max);
+  }
+
   /** Basic test */
   public void testBasics() throws Exception {
     Directory dir = newDirectory();
@@ -88,7 +93,7 @@ public class TestDoubleRangeFieldQueries extends BaseRangeFieldQueryTestCase {
     document.add(new DoubleRangeField(FIELD_NAME, new double[] {10.0, -10.0}, new double[] {20.0, 10.0}));
     writer.addDocument(document);
 
-    // intersects (contains)
+    // intersects (contains, crosses)
     document = new Document();
     document.add(new DoubleRangeField(FIELD_NAME, new double[] {-20.0, -20.0}, new double[] {30.0, 30.1}));
     writer.addDocument(document);
@@ -127,6 +132,8 @@ public class TestDoubleRangeFieldQueries extends BaseRangeFieldQueryTestCase {
         new double[] {-11.0, -15.0}, new double[] {15.0, 20.0})));
     assertEquals(2, searcher.count(DoubleRangeField.newContainsQuery(FIELD_NAME,
         new double[] {-11.0, -15.0}, new double[] {15.0, 20.0})));
+    assertEquals(5, searcher.count(DoubleRangeField.newCrossesQuery(FIELD_NAME,
+        new double[] {-11.0, -15.0}, new double[] {15.0, 20.0})));
 
     reader.close();
     writer.close();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6cbda026/lucene/sandbox/src/test/org/apache/lucene/search/TestFloatRangeFieldQueries.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/TestFloatRangeFieldQueries.java b/lucene/sandbox/src/test/org/apache/lucene/search/TestFloatRangeFieldQueries.java
index 1980782..3509e35 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/search/TestFloatRangeFieldQueries.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/TestFloatRangeFieldQueries.java
@@ -73,6 +73,11 @@ public class TestFloatRangeFieldQueries extends BaseRangeFieldQueryTestCase {
     return FloatRangeField.newWithinQuery(FIELD_NAME, ((FloatRange)r).min, ((FloatRange)r).max);
   }
 
+  @Override
+  protected Query newCrossesQuery(Range r) {
+    return FloatRangeField.newCrossesQuery(FIELD_NAME, ((FloatRange)r).min, ((FloatRange)r).max);
+  }
+
   /** Basic test */
   public void testBasics() throws Exception {
     Directory dir = newDirectory();
@@ -88,7 +93,7 @@ public class TestFloatRangeFieldQueries extends BaseRangeFieldQueryTestCase {
     document.add(new FloatRangeField(FIELD_NAME, new float[] {10.0f, -10.0f}, new float[] {20.0f, 10.0f}));
     writer.addDocument(document);
 
-    // intersects (contains)
+    // intersects (contains, crosses)
     document = new Document();
     document.add(new FloatRangeField(FIELD_NAME, new float[] {-20.0f, -20.0f}, new float[] {30.0f, 30.1f}));
     writer.addDocument(document);
@@ -127,6 +132,8 @@ public class TestFloatRangeFieldQueries extends BaseRangeFieldQueryTestCase {
         new float[] {-11.0f, -15.0f}, new float[] {15.0f, 20.0f})));
     assertEquals(2, searcher.count(FloatRangeField.newContainsQuery(FIELD_NAME,
         new float[] {-11.0f, -15.0f}, new float[] {15.0f, 20.0f})));
+    assertEquals(5, searcher.count(FloatRangeField.newCrossesQuery(FIELD_NAME,
+        new float[] {-11.0f, -15.0f}, new float[] {15.0f, 20.0f})));
 
     reader.close();
     writer.close();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6cbda026/lucene/sandbox/src/test/org/apache/lucene/search/TestIntRangeFieldQueries.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/TestIntRangeFieldQueries.java b/lucene/sandbox/src/test/org/apache/lucene/search/TestIntRangeFieldQueries.java
index 64da229..0bb782e 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/search/TestIntRangeFieldQueries.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/TestIntRangeFieldQueries.java
@@ -73,6 +73,11 @@ public class TestIntRangeFieldQueries extends BaseRangeFieldQueryTestCase {
     return IntRangeField.newWithinQuery(FIELD_NAME, ((IntRange)r).min, ((IntRange)r).max);
   }
 
+  @Override
+  protected Query newCrossesQuery(Range r) {
+    return IntRangeField.newCrossesQuery(FIELD_NAME, ((IntRange)r).min, ((IntRange)r).max);
+  }
+
   /** Basic test */
   public void testBasics() throws Exception {
     Directory dir = newDirectory();
@@ -88,7 +93,7 @@ public class TestIntRangeFieldQueries extends BaseRangeFieldQueryTestCase {
     document.add(new IntRangeField(FIELD_NAME, new int[] {10, -10}, new int[] {20, 10}));
     writer.addDocument(document);
 
-    // intersects (contains)
+    // intersects (contains / crosses)
     document = new Document();
     document.add(new IntRangeField(FIELD_NAME, new int[] {-20, -20}, new int[] {30, 30}));
     writer.addDocument(document);
@@ -127,6 +132,8 @@ public class TestIntRangeFieldQueries extends BaseRangeFieldQueryTestCase {
         new int[] {-11, -15}, new int[] {15, 20})));
     assertEquals(2, searcher.count(IntRangeField.newContainsQuery(FIELD_NAME,
         new int[] {-11, -15}, new int[] {15, 20})));
+    assertEquals(4, searcher.count(IntRangeField.newCrossesQuery(FIELD_NAME,
+        new int[] {-11, -15}, new int[] {15, 20})));
 
     reader.close();
     writer.close();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6cbda026/lucene/sandbox/src/test/org/apache/lucene/search/TestLongRangeFieldQueries.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/TestLongRangeFieldQueries.java b/lucene/sandbox/src/test/org/apache/lucene/search/TestLongRangeFieldQueries.java
index 0f947ff..fc21a64 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/search/TestLongRangeFieldQueries.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/TestLongRangeFieldQueries.java
@@ -73,6 +73,11 @@ public class TestLongRangeFieldQueries extends BaseRangeFieldQueryTestCase {
     return LongRangeField.newWithinQuery(FIELD_NAME, ((LongRange)r).min, ((LongRange)r).max);
   }
 
+  @Override
+  protected Query newCrossesQuery(Range r) {
+    return LongRangeField.newCrossesQuery(FIELD_NAME, ((LongRange)r).min, ((LongRange)r).max);
+  }
+
   /** Basic test */
   public void testBasics() throws Exception {
     Directory dir = newDirectory();
@@ -88,7 +93,7 @@ public class TestLongRangeFieldQueries extends BaseRangeFieldQueryTestCase {
     document.add(new LongRangeField(FIELD_NAME, new long[] {10, -10}, new long[] {20, 10}));
     writer.addDocument(document);
 
-    // intersects (contains)
+    // intersects (contains, crosses)
     document = new Document();
     document.add(new LongRangeField(FIELD_NAME, new long[] {-20, -20}, new long[] {30, 30}));
     writer.addDocument(document);
@@ -127,6 +132,8 @@ public class TestLongRangeFieldQueries extends BaseRangeFieldQueryTestCase {
         new long[] {-11, -15}, new long[] {15, 20})));
     assertEquals(2, searcher.count(LongRangeField.newContainsQuery(FIELD_NAME,
         new long[] {-11, -15}, new long[] {15, 20})));
+    assertEquals(4, searcher.count(LongRangeField.newCrossesQuery(FIELD_NAME,
+        new long[] {-11, -15}, new long[] {15, 20})));
 
     reader.close();
     writer.close();