You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ro...@apache.org on 2015/05/20 15:47:19 UTC

svn commit: r1680569 [2/2] - in /lucene/dev/branches/branch_5x: ./ lucene/ lucene/core/ lucene/core/src/java/org/apache/lucene/search/payloads/ lucene/core/src/java/org/apache/lucene/search/spans/ lucene/core/src/test/org/apache/lucene/search/spans/ lu...

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanWithinQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanWithinQuery.java?rev=1680569&r1=1680568&r2=1680569&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanWithinQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanWithinQuery.java Wed May 20 13:47:18 2015
@@ -18,16 +18,15 @@ package org.apache.lucene.search.spans;
  */
 
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.index.TermContext;
+import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.util.Bits;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Map;
 
 /** Keep matches that are contained within another Spans. */
 public class SpanWithinQuery extends SpanContainQuery {
+
   /** Construct a SpanWithinQuery matching spans from <code>little</code>
    * that are inside of <code>big</code>.
    * This query has the boost of <code>little</code>.
@@ -49,62 +48,79 @@ public class SpanWithinQuery extends Spa
           (SpanQuery) little.clone());
   }
 
-  /** 
-   * Return spans from <code>little</code> that are contained in a spans from <code>big</code>.
-   * The payload is from the spans of <code>little</code>.
-   */
   @Override
-  public Spans getSpans(final LeafReaderContext context, final Bits acceptDocs, final Map<Term,TermContext> termContexts, SpanCollector collector) throws IOException {
-    ArrayList<Spans> containerContained = prepareConjunction(context, acceptDocs, termContexts, collector);
-    if (containerContained == null) {
-      return null;
-    }
+  public SpanWeight createWeight(IndexSearcher searcher, boolean needsScores, SpanCollectorFactory factory) throws IOException {
+    SpanWeight bigWeight = big.createWeight(searcher, false, factory);
+    SpanWeight littleWeight = little.createWeight(searcher, false, factory);
+    SpanSimilarity similarity = SpanSimilarity.build(this, searcher, needsScores, bigWeight, littleWeight);
+    return new SpanWithinWeight(similarity, factory, bigWeight, littleWeight);
+  }
 
-    Spans big = containerContained.get(0);
-    Spans little = containerContained.get(1);
+  public class SpanWithinWeight extends SpanContainWeight {
 
-    return new ContainSpans(big, little, little) {
+    public SpanWithinWeight(SpanSimilarity similarity, SpanCollectorFactory factory,
+                            SpanWeight bigWeight, SpanWeight littleWeight) throws IOException {
+      super(similarity, factory, bigWeight, littleWeight);
+    }
 
-      @Override
-      boolean twoPhaseCurrentDocMatches() throws IOException {
-        oneExhaustedInCurrentDoc = false;
-        assert littleSpans.startPosition() == -1;
-        while (littleSpans.nextStartPosition() != NO_MORE_POSITIONS) {
-          while (bigSpans.endPosition() < littleSpans.endPosition()) {
-            if (bigSpans.nextStartPosition() == NO_MORE_POSITIONS) {
-              oneExhaustedInCurrentDoc = true;
-              return false;
-            }
-          }
-          if (bigSpans.startPosition() <= littleSpans.startPosition()) {
-            atFirstInCurrentDoc = true;
-            return true;
-          }
-        } 
-        oneExhaustedInCurrentDoc = true;
-        return false;
+    /**
+     * Return spans from <code>little</code> that are contained in a spans from <code>big</code>.
+     * The payload is from the spans of <code>little</code>.
+     */
+    @Override
+    public Spans getSpans(final LeafReaderContext context, final Bits acceptDocs, SpanCollector collector) throws IOException {
+      ArrayList<Spans> containerContained = prepareConjunction(context, acceptDocs, collector);
+      if (containerContained == null) {
+        return null;
       }
 
-      @Override
-      public int nextStartPosition() throws IOException {
-        if (atFirstInCurrentDoc) {
-          atFirstInCurrentDoc = false;
-          return littleSpans.startPosition();
-        }
-        while (littleSpans.nextStartPosition() != NO_MORE_POSITIONS) {
-          while (bigSpans.endPosition() < littleSpans.endPosition()) {
-            if (bigSpans.nextStartPosition() == NO_MORE_POSITIONS) {
-              oneExhaustedInCurrentDoc = true;
-              return NO_MORE_POSITIONS;
+      Spans big = containerContained.get(0);
+      Spans little = containerContained.get(1);
+
+      return new ContainSpans(big, little, little) {
+
+        @Override
+        boolean twoPhaseCurrentDocMatches() throws IOException {
+          oneExhaustedInCurrentDoc = false;
+          assert littleSpans.startPosition() == -1;
+          while (littleSpans.nextStartPosition() != NO_MORE_POSITIONS) {
+            while (bigSpans.endPosition() < littleSpans.endPosition()) {
+              if (bigSpans.nextStartPosition() == NO_MORE_POSITIONS) {
+                oneExhaustedInCurrentDoc = true;
+                return false;
+              }
+            }
+            if (bigSpans.startPosition() <= littleSpans.startPosition()) {
+              atFirstInCurrentDoc = true;
+              return true;
             }
           }
-          if (bigSpans.startPosition() <= littleSpans.startPosition()) {
+          oneExhaustedInCurrentDoc = true;
+          return false;
+        }
+
+        @Override
+        public int nextStartPosition() throws IOException {
+          if (atFirstInCurrentDoc) {
+            atFirstInCurrentDoc = false;
             return littleSpans.startPosition();
           }
+          while (littleSpans.nextStartPosition() != NO_MORE_POSITIONS) {
+            while (bigSpans.endPosition() < littleSpans.endPosition()) {
+              if (bigSpans.nextStartPosition() == NO_MORE_POSITIONS) {
+                oneExhaustedInCurrentDoc = true;
+                return NO_MORE_POSITIONS;
+              }
+            }
+            if (bigSpans.startPosition() <= littleSpans.startPosition()) {
+              return littleSpans.startPosition();
+            }
+          }
+          oneExhaustedInCurrentDoc = true;
+          return NO_MORE_POSITIONS;
         }
-        oneExhaustedInCurrentDoc = true;
-        return NO_MORE_POSITIONS;
-      }
-    };
+      };
+    }
   }
+
 }
\ No newline at end of file

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/spans/JustCompileSearchSpans.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/spans/JustCompileSearchSpans.java?rev=1680569&r1=1680568&r2=1680569&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/spans/JustCompileSearchSpans.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/spans/JustCompileSearchSpans.java Wed May 20 13:47:18 2015
@@ -17,15 +17,10 @@ package org.apache.lucene.search.spans;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.index.TermContext;
+import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.similarities.Similarity;
-import org.apache.lucene.util.Bits;
 
 import java.io.IOException;
-import java.util.Map;
-import java.util.Set;
 
 /**
  * Holds all implementations of classes in the o.a.l.s.spans package as a
@@ -84,17 +79,12 @@ final class JustCompileSearchSpans {
   static final class JustCompileSpanQuery extends SpanQuery {
 
     @Override
-    protected void extractTerms(Set<Term> terms) {
-      throw new UnsupportedOperationException(UNSUPPORTED_MSG);
-    }
-
-    @Override
     public String getField() {
       throw new UnsupportedOperationException(UNSUPPORTED_MSG);
     }
 
     @Override
-    public Spans getSpans(LeafReaderContext context, Bits acceptDocs, Map<Term,TermContext> termContexts, SpanCollector collector) {
+    public SpanWeight createWeight(IndexSearcher searcher, boolean needsScores, SpanCollectorFactory factory) throws IOException {
       throw new UnsupportedOperationException(UNSUPPORTED_MSG);
     }
 

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/spans/MultiSpansWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/spans/MultiSpansWrapper.java?rev=1680569&r1=1680568&r2=1680569&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/spans/MultiSpansWrapper.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/spans/MultiSpansWrapper.java Wed May 20 13:47:18 2015
@@ -21,14 +21,10 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.SlowCompositeReaderWrapper;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.index.TermContext;
+import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.util.Bits;
 
 import java.io.IOException;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
 
 /**
  * 
@@ -44,17 +40,14 @@ public class MultiSpansWrapper {
   }
 
   public static Spans wrap(IndexReader reader, SpanQuery spanQuery, SpanCollector collector) throws IOException {
+
+    IndexSearcher searcher = new IndexSearcher(reader);
+    searcher.setQueryCache(null);
     LeafReader lr = SlowCompositeReaderWrapper.wrap(reader); // slow, but ok for testing
     LeafReaderContext lrContext = lr.getContext();
-    SpanQuery rewrittenQuery = (SpanQuery) spanQuery.rewrite(lr); // get the term contexts so getSpans can be called directly
-    HashSet<Term> termSet = new HashSet<>();
-    rewrittenQuery.extractTerms(termSet);
-    Map<Term,TermContext> termContexts = new HashMap<>();
-    for (Term term: termSet) {
-      TermContext termContext = TermContext.build(lrContext, term);
-      termContexts.put(term, termContext);
-    }
-    Spans actSpans = spanQuery.getSpans(lrContext, new Bits.MatchAllBits(lr.numDocs()), termContexts, collector);
-    return actSpans;
+
+    SpanWeight w = (SpanWeight) searcher.createNormalizedWeight(spanQuery, false);
+
+    return w.getSpans(lrContext, new Bits.MatchAllBits(lr.numDocs()), collector);
   }
 }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/spans/TestFieldMaskingSpanQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/spans/TestFieldMaskingSpanQuery.java?rev=1680569&r1=1680568&r2=1680569&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/spans/TestFieldMaskingSpanQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/spans/TestFieldMaskingSpanQuery.java Wed May 20 13:47:18 2015
@@ -17,9 +17,6 @@ package org.apache.lucene.search.spans;
  * limitations under the License.
  */
 
-import java.util.HashSet;
-import java.util.Set;
-
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
@@ -36,7 +33,11 @@ import org.apache.lucene.util.LuceneTest
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 
-import static org.apache.lucene.search.spans.SpanTestUtil.*;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.lucene.search.spans.SpanTestUtil.assertFinished;
+import static org.apache.lucene.search.spans.SpanTestUtil.assertNext;
 
 public class TestFieldMaskingSpanQuery extends LuceneTestCase {
 
@@ -141,7 +142,7 @@ public class TestFieldMaskingSpanQuery e
     QueryUtils.checkEqual(q, qr);
 
     Set<Term> terms = new HashSet<>();
-    qr.extractTerms(terms);
+    qr.createWeight(searcher, false).extractTerms(terms);
     assertEquals(1, terms.size());
   }
   
@@ -161,7 +162,7 @@ public class TestFieldMaskingSpanQuery e
     QueryUtils.checkUnequal(q, qr);
 
     Set<Term> terms = new HashSet<>();
-    qr.extractTerms(terms);
+    qr.createWeight(searcher, false).extractTerms(terms);
     assertEquals(2, terms.size());
   }
   

Modified: lucene/dev/branches/branch_5x/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java?rev=1680569&r1=1680568&r2=1680569&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java (original)
+++ lucene/dev/branches/branch_5x/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java Wed May 20 13:47:18 2015
@@ -29,7 +29,6 @@ import org.apache.lucene.index.LeafReade
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.index.TermContext;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.memory.MemoryIndex;
 import org.apache.lucene.queries.CommonTermsQuery;
@@ -68,7 +67,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.TreeSet;
 
 
 /**
@@ -301,15 +299,9 @@ public class WeightedSpanTermExtractor {
         q = spanQuery;
       }
       LeafReaderContext context = getLeafContext();
-      Map<Term,TermContext> termContexts = new HashMap<>();
-      TreeSet<Term> extractedTerms = new TreeSet<>();
       SpanWeight w = (SpanWeight) searcher.createNormalizedWeight(q, false);
-      w.extractTerms(extractedTerms);
-      for (Term term : extractedTerms) {
-        termContexts.put(term, TermContext.build(context, term));
-      }
       Bits acceptDocs = context.reader().getLiveDocs();
-      final Spans spans = q.getSpans(context, acceptDocs, termContexts, w.getSpanCollectorFactory().newCollector());
+      final Spans spans = w.getSpans(context, acceptDocs);
       if (spans == null) {
         return;
       }

Modified: lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/search/spans/AssertingSpanQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/search/spans/AssertingSpanQuery.java?rev=1680569&r1=1680568&r2=1680569&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/search/spans/AssertingSpanQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/search/spans/AssertingSpanQuery.java Wed May 20 13:47:18 2015
@@ -18,16 +18,10 @@ package org.apache.lucene.search.spans;
  */
 
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.index.TermContext;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
-import org.apache.lucene.util.Bits;
 
 import java.io.IOException;
-import java.util.Map;
-import java.util.Set;
 
 /** Wraps a span query with asserts */
 public class AssertingSpanQuery extends SpanQuery {
@@ -38,21 +32,6 @@ public class AssertingSpanQuery extends
   }
 
   @Override
-  protected void extractTerms(Set<Term> terms) {
-    in.extractTerms(terms);
-  }
-
-  @Override
-  public Spans getSpans(LeafReaderContext context, Bits acceptDocs, Map<Term,TermContext> termContexts, SpanCollector collector) throws IOException {
-    Spans spans = in.getSpans(context, acceptDocs, termContexts, collector);
-    if (spans == null) {
-      return null;
-    } else {
-      return new AssertingSpans(spans);
-    }
-  }
-
-  @Override
   public String getField() {
     return in.getField();
   }
@@ -63,15 +42,9 @@ public class AssertingSpanQuery extends
   }
 
   @Override
-  public SpanWeight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
-    // TODO: we are wasteful and createWeight twice in this case... use VirtualMethod?
-    // we need to not wrap if the query is e.g. a Payload one that overrides this (it should really be final)
-    SpanWeight weight = in.createWeight(searcher, needsScores);
-    if (weight.getClass() == SpanWeight.class) {
-      return super.createWeight(searcher, needsScores);
-    } else {
-      return weight;
-    }
+  public SpanWeight createWeight(IndexSearcher searcher, boolean needsScores, SpanCollectorFactory factory) throws IOException {
+    SpanWeight weight = in.createWeight(searcher, needsScores, factory);
+    return new AssertingSpanWeight(weight);
   }
 
   @Override