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 2019/08/07 10:58:50 UTC

[lucene-solr] 01/02: LUCENE-8941: Build wildcard matches lazily

This is an automated email from the ASF dual-hosted git repository.

romseygeek pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit b5b78e0adeb9db9345b69abedabd8c5cd684df7b
Author: Alan Woodward <ro...@apache.org>
AuthorDate: Mon Aug 5 09:44:44 2019 +0100

    LUCENE-8941: Build wildcard matches lazily
---
 lucene/CHANGES.txt                                 |  3 +
 .../lucene/search/DisjunctionMatchesIterator.java  | 84 ++++++++++++++++++++--
 .../apache/lucene/search/TestMatchesIterator.java  | 66 ++++++++++++++++-
 3 files changed, 148 insertions(+), 5 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index bd1ee2b..88f22f1 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -42,6 +42,9 @@ Optimizations
 * LUCENE-8935: BooleanQuery with no scoring clause can now early terminate the query when
 the total hits is not requested.
 
+* LUCENE-8941: Matches on wildcard queries will defer building their full
+  disjunction until a MatchesIterator is pulled (Alan Woodward)
+
 Other
 
 * LUCENE-8778 LUCENE-8911: Define analyzer SPI names as static final fields and document the names in Javadocs.
diff --git a/lucene/core/src/java/org/apache/lucene/search/DisjunctionMatchesIterator.java b/lucene/core/src/java/org/apache/lucene/search/DisjunctionMatchesIterator.java
index f28c051..986b8d9 100644
--- a/lucene/core/src/java/org/apache/lucene/search/DisjunctionMatchesIterator.java
+++ b/lucene/core/src/java/org/apache/lucene/search/DisjunctionMatchesIterator.java
@@ -74,7 +74,6 @@ final class DisjunctionMatchesIterator implements MatchesIterator {
    */
   static MatchesIterator fromTermsEnum(LeafReaderContext context, int doc, Query query, String field, BytesRefIterator terms) throws IOException {
     Objects.requireNonNull(field);
-    List<MatchesIterator> mis = new ArrayList<>();
     Terms t = context.reader().terms(field);
     if (t == null)
       return null;
@@ -84,15 +83,92 @@ final class DisjunctionMatchesIterator implements MatchesIterator {
       if (te.seekExact(term)) {
         PostingsEnum pe = te.postings(reuse, PostingsEnum.OFFSETS);
         if (pe.advance(doc) == doc) {
-          mis.add(new TermMatchesIterator(query, pe));
-          reuse = null;
+          return new TermsEnumDisjunctionMatchesIterator(new TermMatchesIterator(query, pe), terms, te, doc, query);
         }
         else {
           reuse = pe;
         }
       }
     }
-    return fromSubIterators(mis);
+    return null;
+  }
+
+  // MatchesIterator over a set of terms that only loads the first matching term at construction,
+  // waiting until the iterator is actually used before it loads all other matching terms.
+  private static class TermsEnumDisjunctionMatchesIterator implements MatchesIterator {
+
+    private final MatchesIterator first;
+    private final BytesRefIterator terms;
+    private final TermsEnum te;
+    private final int doc;
+    private final Query query;
+
+    private MatchesIterator it = null;
+
+    TermsEnumDisjunctionMatchesIterator(MatchesIterator first, BytesRefIterator terms, TermsEnum te, int doc, Query query) {
+      this.first = first;
+      this.terms = terms;
+      this.te = te;
+      this.doc = doc;
+      this.query = query;
+    }
+
+    private void init() throws IOException {
+      List<MatchesIterator> mis = new ArrayList<>();
+      mis.add(first);
+      PostingsEnum reuse = null;
+      for (BytesRef term = terms.next(); term != null; term = terms.next()) {
+        if (te.seekExact(term)) {
+          PostingsEnum pe = te.postings(reuse, PostingsEnum.OFFSETS);
+          if (pe.advance(doc) == doc) {
+            mis.add(new TermMatchesIterator(query, pe));
+            reuse = null;
+          } else {
+            reuse = pe;
+          }
+        }
+      }
+      it = fromSubIterators(mis);
+    }
+
+    @Override
+    public boolean next() throws IOException {
+      if (it == null) {
+        init();
+      }
+      assert it != null;
+      return it.next();
+    }
+
+    @Override
+    public int startPosition() {
+      return it.startPosition();
+    }
+
+    @Override
+    public int endPosition() {
+      return it.endPosition();
+    }
+
+    @Override
+    public int startOffset() throws IOException {
+      return it.startOffset();
+    }
+
+    @Override
+    public int endOffset() throws IOException {
+      return it.endOffset();
+    }
+
+    @Override
+    public MatchesIterator getSubMatches() throws IOException {
+      return it.getSubMatches();
+    }
+
+    @Override
+    public Query getQuery() {
+      return it.getQuery();
+    }
   }
 
   static MatchesIterator fromSubIterators(List<MatchesIterator> mis) throws IOException {
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestMatchesIterator.java b/lucene/core/src/test/org/apache/lucene/search/TestMatchesIterator.java
index 45fda8c..235bead 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestMatchesIterator.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestMatchesIterator.java
@@ -31,25 +31,30 @@ import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.IntPoint;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.FilterLeafReader;
 import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.ReaderUtil;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.spans.SpanNearQuery;
 import org.apache.lucene.search.spans.SpanOrQuery;
 import org.apache.lucene.search.spans.SpanQuery;
 import org.apache.lucene.search.spans.SpanTermQuery;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 
 public class TestMatchesIterator extends LuceneTestCase {
 
   protected IndexSearcher searcher;
   protected Directory directory;
-  protected IndexReader reader;
+  protected IndexReader reader = null;
 
   private static final String FIELD_WITH_OFFSETS = "field_offsets";
   private static final String FIELD_NO_OFFSETS = "field_no_offsets";
@@ -701,4 +706,63 @@ public class TestMatchesIterator extends LuceneTestCase {
     });
   }
 
+  public void testMinimalSeekingWithWildcards() throws IOException {
+    SeekCountingLeafReader reader = new SeekCountingLeafReader(getOnlyLeafReader(this.reader));
+    this.searcher = new IndexSearcher(reader);
+    Query query = new PrefixQuery(new Term(FIELD_WITH_OFFSETS, "w"));
+    Weight w = searcher.createWeight(query.rewrite(reader), ScoreMode.COMPLETE, 1);
+
+    // docs 0-3 match several different terms here, but we only seek to the first term and
+    // then short-cut return; other terms are ignored until we try and iterate over matches
+    int[] expectedSeeks = new int[]{ 1, 1, 1, 1, 6, 6 };
+    int i = 0;
+    for (LeafReaderContext ctx : reader.leaves()) {
+      for (int doc = 0; doc < ctx.reader().maxDoc(); doc++) {
+        reader.seeks = 0;
+        w.matches(ctx, doc);
+        assertEquals("Unexpected seek count on doc " + doc, expectedSeeks[i], reader.seeks);
+        i++;
+      }
+    }
+  }
+
+  private static class SeekCountingLeafReader extends FilterLeafReader {
+
+    int seeks = 0;
+
+    public SeekCountingLeafReader(LeafReader in) {
+      super(in);
+    }
+
+    @Override
+    public Terms terms(String field) throws IOException {
+      Terms terms = super.terms(field);
+      if (terms == null) {
+        return null;
+      }
+      return new FilterTerms(terms) {
+        @Override
+        public TermsEnum iterator() throws IOException {
+          return new FilterTermsEnum(super.iterator()) {
+            @Override
+            public boolean seekExact(BytesRef text) throws IOException {
+              seeks++;
+              return super.seekExact(text);
+            }
+          };
+        }
+      };
+    }
+
+    @Override
+    public CacheHelper getCoreCacheHelper() {
+      return null;
+    }
+
+    @Override
+    public CacheHelper getReaderCacheHelper() {
+      return null;
+    }
+  }
+
 }