You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2015/04/29 13:58:37 UTC

svn commit: r1676723 - in /lucene/dev/branches/branch_5x: ./ lucene/ lucene/core/ lucene/core/src/java/org/apache/lucene/search/spans/ lucene/core/src/test/org/apache/lucene/search/spans/ lucene/test-framework/ lucene/test-framework/src/java/org/apache...

Author: rmuir
Date: Wed Apr 29 11:58:34 2015
New Revision: 1676723

URL: http://svn.apache.org/r1676723
Log:
LUCENE-6083: SpanWithinQuery and SpanContainingQuery

Added:
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/ConjunctionSpans.java
      - copied unchanged from r1676716, lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/spans/ConjunctionSpans.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/ContainSpans.java
      - copied unchanged from r1676716, lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/spans/ContainSpans.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanContainQuery.java
      - copied unchanged from r1676716, lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/spans/SpanContainQuery.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanContainingQuery.java
      - copied unchanged from r1676716, lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/spans/SpanContainingQuery.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanWithinQuery.java
      - copied unchanged from r1676716, lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/spans/SpanWithinQuery.java
    lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/spans/TestSpanContainQuery.java
      - copied unchanged from r1676716, lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/spans/TestSpanContainQuery.java
Modified:
    lucene/dev/branches/branch_5x/   (props changed)
    lucene/dev/branches/branch_5x/lucene/   (props changed)
    lucene/dev/branches/branch_5x/lucene/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_5x/lucene/core/   (props changed)
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/NearSpans.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/package-info.java
    lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/spans/TestSpanSearchEquivalence.java
    lucene/dev/branches/branch_5x/lucene/test-framework/   (props changed)
    lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/search/spans/SpanTestUtil.java

Modified: lucene/dev/branches/branch_5x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/CHANGES.txt?rev=1676723&r1=1676722&r2=1676723&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_5x/lucene/CHANGES.txt Wed Apr 29 11:58:34 2015
@@ -56,6 +56,9 @@ New Features
 * LUCENE-6451: Expressions now support bindings keys that look like
   zero arg functions (Jack Conradson via Ryan Ernst)
 
+* LUCENE-6083: Add SpanWithinQuery and SpanContainingQuery that return
+  spans inside of / containing another spans. (Paul Elschot via Robert Muir)
+
 Optimizations
 
 * LUCENE-6379: IndexWriter.deleteDocuments(Query...) now detects if

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/NearSpans.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/NearSpans.java?rev=1676723&r1=1676722&r2=1676723&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/NearSpans.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/NearSpans.java Wed Apr 29 11:58:34 2015
@@ -17,82 +17,18 @@ package org.apache.lucene.search.spans;
  * limitations under the License.
  */
 
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.ConjunctionDISI;
-import org.apache.lucene.search.TwoPhaseIterator;
-
-import java.io.IOException;
 import java.util.List;
-import java.util.Objects;
 
 /**
- * Common super class for un/ordered Spans
+ * Common super class for un/ordered Spans with a maximum slop between them.
  */
-abstract class NearSpans extends Spans {
+abstract class NearSpans extends ConjunctionSpans {
   final SpanNearQuery query;
   final int allowedSlop;
 
-  final Spans[] subSpans; // in query order
-  final DocIdSetIterator conjunction; // use to move to next doc with all clauses
-  boolean atFirstInCurrentDoc;
-  boolean oneExhaustedInCurrentDoc; // no more results possbile in current doc
-
-  NearSpans(SpanNearQuery query, List<Spans> subSpans)
-  throws IOException {
-    this.query = Objects.requireNonNull(query);
+  NearSpans(SpanNearQuery query, List<Spans> subSpans) {
+    super(subSpans);
+    this.query = query;
     this.allowedSlop = query.getSlop();
-    if (subSpans.size() < 2) {
-      throw new IllegalArgumentException("Less than 2 subSpans: " + query);
-    }
-    this.subSpans = subSpans.toArray(new Spans[subSpans.size()]); // in query order
-    this.conjunction = ConjunctionDISI.intersect(subSpans);
-  }
-
-  @Override
-  public int docID() {
-    return conjunction.docID();
-  }
-
-  @Override
-  public long cost() {
-    return conjunction.cost();
-  }
-
-  @Override
-  public int nextDoc() throws IOException {
-    return (conjunction.nextDoc() == NO_MORE_DOCS)
-            ? NO_MORE_DOCS
-            : toMatchDoc();
-  }
-
-  @Override
-  public int advance(int target) throws IOException {
-    return (conjunction.advance(target) == NO_MORE_DOCS)
-            ? NO_MORE_DOCS
-            : toMatchDoc();
   }
-
-  abstract int toMatchDoc() throws IOException;
-
-  abstract boolean twoPhaseCurrentDocMatches() throws IOException;
-
-  /**
-   * Return a {@link TwoPhaseIterator} view of this {@link NearSpans}.
-   */
-  @Override
-  public TwoPhaseIterator asTwoPhaseIterator() {
-    TwoPhaseIterator res = new TwoPhaseIterator(conjunction) {
-
-      @Override
-      public boolean matches() throws IOException {
-        return twoPhaseCurrentDocMatches();
-      }
-    };
-    return res;
-  }
-
-  public Spans[] getSubSpans() {
-    return subSpans;
-  }
-
 }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java?rev=1676723&r1=1676722&r2=1676723&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java Wed Apr 29 11:58:34 2015
@@ -56,35 +56,6 @@ public class NearSpansOrdered extends Ne
     this.atFirstInCurrentDoc = true; // -1 startPosition/endPosition also at doc -1
   }
 
-  /** Advances the subSpans to just after an ordered match with a minimum slop
-   * that is smaller than the slop allowed by the SpanNearQuery.
-   * @return true iff there is such a match.
-   */
-  @Override
-  int toMatchDoc() throws IOException {
-    subSpansToFirstStartPosition();
-    while (true) {
-      if (! stretchToOrder()) {
-        if (conjunction.nextDoc() == NO_MORE_DOCS) {
-          return NO_MORE_DOCS;
-        }
-        subSpansToFirstStartPosition();
-      } else {
-        if (shrinkToAfterShortestMatch()) {
-          atFirstInCurrentDoc = true;
-          return conjunction.docID();
-        }
-        // not a match, after shortest ordered spans, not at beginning of doc.
-        if (oneExhaustedInCurrentDoc) {
-          if (conjunction.nextDoc() == NO_MORE_DOCS) {
-            return NO_MORE_DOCS;
-          }
-          subSpansToFirstStartPosition();
-        }
-      }
-    }
-  }
-
   @Override
   boolean twoPhaseCurrentDocMatches() throws IOException {
     subSpansToFirstStartPosition();

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java?rev=1676723&r1=1676722&r2=1676723&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java Wed Apr 29 11:58:34 2015
@@ -192,30 +192,6 @@ public class NearSpansUnordered extends
   }
 
   @Override
-  int toMatchDoc() throws IOException {
-    // at doc with all subSpans
-    subSpanCellsToPositionQueue();
-    while (true) {
-      if (atMatch()) {
-        atFirstInCurrentDoc = true;
-        oneExhaustedInCurrentDoc = false;
-        return conjunction.docID();
-      }
-      assert minPositionCell().startPosition() != NO_MORE_POSITIONS;
-      if (minPositionCell().nextStartPosition() != NO_MORE_POSITIONS) {
-        spanPositionQueue.updateTop();
-      }
-      else { // exhausted a subSpan in current doc
-        if (conjunction.nextDoc() == NO_MORE_DOCS) {
-          return NO_MORE_DOCS;
-        }
-        // at doc with all subSpans
-        subSpanCellsToPositionQueue();
-      }
-    }
-  }
-
-  @Override
   boolean twoPhaseCurrentDocMatches() throws IOException {
     // at doc with all subSpans
     subSpanCellsToPositionQueue();

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/package-info.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/package-info.java?rev=1676723&r1=1676722&r2=1676723&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/package-info.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/package-info.java Wed Apr 29 11:58:34 2015
@@ -35,6 +35,12 @@
  * which occur near one another, and can be used to implement things like
  * phrase search (when constructed from {@link org.apache.lucene.search.spans.SpanTermQuery}s)
  * and inter-phrase proximity (when constructed from other {@link org.apache.lucene.search.spans.SpanNearQuery}s).</li>
+ *
+ * <li> A {@link org.apache.lucene.search.spans.SpanWithinQuery SpanWithinQuery} matches spans
+ * which occur inside of another spans. </li>
+ *
+ * <li> A {@link org.apache.lucene.search.spans.SpanContainingQuery SpanContainingQuery} matches spans
+ * which contain another spans. </li>
  * 
  * <li>A {@link org.apache.lucene.search.spans.SpanOrQuery SpanOrQuery} merges spans from a
  * number of other {@link org.apache.lucene.search.spans.SpanQuery}s.</li>

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/spans/TestSpanSearchEquivalence.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/spans/TestSpanSearchEquivalence.java?rev=1676723&r1=1676722&r2=1676723&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/spans/TestSpanSearchEquivalence.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/spans/TestSpanSearchEquivalence.java Wed Apr 29 11:58:34 2015
@@ -419,4 +419,37 @@ public class TestSpanSearchEquivalence e
     Query q2 = nearQuery;
     assertSameSet(q1, q2);
   }
+  
+  /** SpanWithinQuery(A, B) ⊆ SpanNearQuery(A) */
+  public void testSpanWithinVsNear() throws Exception {
+    Term t1 = randomTerm();
+    Term t2 = randomTerm();
+    SpanQuery subquery[] = new SpanQuery[] { 
+        spanQuery(new SpanTermQuery(t1)), 
+        spanQuery(new SpanTermQuery(t2)) 
+      };
+    SpanQuery nearQuery = spanQuery(new SpanNearQuery(subquery, 10, true));
+    
+    Term t3 = randomTerm();
+    SpanQuery termQuery = spanQuery(new SpanTermQuery(t3));
+    Query q1 = spanQuery(new SpanWithinQuery(nearQuery, termQuery));
+    assertSubsetOf(q1, termQuery);
+  }
+  
+  /** SpanWithinQuery(A, B) = SpanContainingQuery(A, B) */
+  public void testSpanWithinVsContaining() throws Exception {
+    Term t1 = randomTerm();
+    Term t2 = randomTerm();
+    SpanQuery subquery[] = new SpanQuery[] { 
+        spanQuery(new SpanTermQuery(t1)), 
+        spanQuery(new SpanTermQuery(t2)) 
+      };
+    SpanQuery nearQuery = spanQuery(new SpanNearQuery(subquery, 10, true));
+    
+    Term t3 = randomTerm();
+    SpanQuery termQuery = spanQuery(new SpanTermQuery(t3));
+    Query q1 = spanQuery(new SpanWithinQuery(nearQuery, termQuery));
+    Query q2 = spanQuery(new SpanContainingQuery(nearQuery, termQuery));
+    assertSameSet(q1, q2);
+  }
 }

Modified: lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/search/spans/SpanTestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/search/spans/SpanTestUtil.java?rev=1676723&r1=1676722&r2=1676723&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/search/spans/SpanTestUtil.java (original)
+++ lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/search/spans/SpanTestUtil.java Wed Apr 29 11:58:34 2015
@@ -90,6 +90,20 @@ public class SpanTestUtil {
   }
   
   /**
+   * Makes a new SpanContainingQuery (with additional asserts).
+   */
+  public static SpanQuery spanContainingQuery(SpanQuery big, SpanQuery little) {
+    return spanQuery(new SpanContainingQuery(big, little));
+  }
+  
+  /**
+   * Makes a new SpanWithinQuery (with additional asserts).
+   */
+  public static SpanQuery spanWithinQuery(SpanQuery big, SpanQuery little) {
+    return spanQuery(new SpanWithinQuery(big, little));
+  }
+  
+  /**
    * Makes a new ordered SpanNearQuery (with additional asserts) from the provided {@code terms}
    */
   public static SpanQuery spanNearOrderedQuery(String field, int slop, String... terms) {