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 2020/04/29 09:32:17 UTC

[lucene-solr] branch branch_8x updated: LUCENE-9349: TermInSetQuery should use consumeMatchingTerms in visit() (#1465)

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


The following commit(s) were added to refs/heads/branch_8x by this push:
     new 4be69c8  LUCENE-9349: TermInSetQuery should use consumeMatchingTerms in visit() (#1465)
4be69c8 is described below

commit 4be69c8540b51f52355f9926cd84f3b3de985180
Author: Alan Woodward <ro...@apache.org>
AuthorDate: Wed Apr 29 10:19:05 2020 +0100

    LUCENE-9349: TermInSetQuery should use consumeMatchingTerms in visit() (#1465)
    
    TermInSetQuery currently iterates through all its prefix-encoded terms
    in order to build an array to pass back to its visitor when visit() is called.
    This seems like a waste, particularly when the visitor is not actually
    consuming the terms (for example, when doing a clause-count check
    before executing a search). This commit changes TermInSetQuery to use
    consumeTermsMatching(), and also changes the signature of this method so
    that it takes a BytesRunAutomaton supplier to allow for lazy instantiation.
---
 lucene/CHANGES.txt                                 |  5 +++
 .../org/apache/lucene/search/IndexSearcher.java    |  1 +
 .../org/apache/lucene/search/QueryVisitor.java     |  5 +--
 .../org/apache/lucene/search/TermInSetQuery.java   | 22 +++++++++---
 .../lucene/util/automaton/CompiledAutomaton.java   |  4 +--
 .../apache/lucene/search/TermInSetQueryTest.java   | 42 ++++++++++++++++++++++
 .../search/uhighlight/MultiTermHighlighting.java   |  5 +--
 7 files changed, 74 insertions(+), 10 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 811c741..03ae2ce 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -23,6 +23,11 @@ API Changes
   interface that offers the basic methods to acquire pending merges, run the merge and do accounting 
   around it. (Simon Willnauer)
 
+* LUCENE-9349: QueryVisitor.consumeTermsMatching() now takes a
+  Supplier<ByteRunAutomaton> to enable queries that build large automata to
+  provide them lazily.  TermsInSetQuery switches to using this method
+  to report matching terms. (Alan Woodward)
+
 New Features
 ---------------------
 (No changes)
diff --git a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
index e993832..5c30e45 100644
--- a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
+++ b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
@@ -47,6 +47,7 @@ import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.store.NIOFSDirectory;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.ThreadInterruptedException;
+import org.apache.lucene.util.automaton.ByteRunAutomaton;
 
 /** Implements search over a single IndexReader.
  *
diff --git a/lucene/core/src/java/org/apache/lucene/search/QueryVisitor.java b/lucene/core/src/java/org/apache/lucene/search/QueryVisitor.java
index 15a0eaf..d79775d 100644
--- a/lucene/core/src/java/org/apache/lucene/search/QueryVisitor.java
+++ b/lucene/core/src/java/org/apache/lucene/search/QueryVisitor.java
@@ -19,6 +19,7 @@ package org.apache.lucene.search;
 
 import java.util.Arrays;
 import java.util.Set;
+import java.util.function.Supplier;
 
 import org.apache.lucene.index.Term;
 import org.apache.lucene.util.automaton.ByteRunAutomaton;
@@ -43,11 +44,11 @@ public abstract class QueryVisitor {
    *
    * @param query     the leaf query
    * @param field     the field queried against
-   * @param automaton an automaton defining which terms match
+   * @param automaton a supplier for an automaton defining which terms match
    *
    * @lucene.experimental
    */
-  public void consumeTermsMatching(Query query, String field, ByteRunAutomaton automaton) {
+  public void consumeTermsMatching(Query query, String field, Supplier<ByteRunAutomaton> automaton) {
     visitLeaf(query); // default impl for backward compatibility
   }
 
diff --git a/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java b/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java
index 9e5648b..e3b5dea 100644
--- a/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java
@@ -44,6 +44,11 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.DocIdSetBuilder;
 import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.automaton.Automata;
+import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.ByteRunAutomaton;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
+import org.apache.lucene.util.automaton.Operations;
 
 /**
  * Specialization for a disjunction over many terms that behaves like a
@@ -127,13 +132,22 @@ public class TermInSetQuery extends Query implements Accountable {
     if (visitor.acceptField(field) == false) {
       return;
     }
-    QueryVisitor v = visitor.getSubVisitor(Occur.SHOULD, this);
-    List<Term> terms = new ArrayList<>();
+    if (termData.size() == 1) {
+      visitor.consumeTerms(this, new Term(field, termData.iterator().next()));
+    }
+    if (termData.size() > 1) {
+      visitor.consumeTermsMatching(this, field, this::asByteRunAutomaton);
+    }
+  }
+
+  private ByteRunAutomaton asByteRunAutomaton() {
     TermIterator iterator = termData.iterator();
+    List<Automaton> automata = new ArrayList<>();
     for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
-      terms.add(new Term(field, BytesRef.deepCopyOf(term)));
+      automata.add(Automata.makeBinary(term));
     }
-    v.consumeTerms(this, terms.toArray(new Term[0]));
+    return new CompiledAutomaton(Operations.union(automata)).runAutomaton;
+
   }
 
   @Override
diff --git a/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java b/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java
index 1c9a235..c14cd91 100644
--- a/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java
+++ b/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java
@@ -354,12 +354,12 @@ public class CompiledAutomaton implements Accountable {
     if (visitor.acceptField(field)) {
       switch (type) {
         case NORMAL:
-          visitor.consumeTermsMatching(parent, field, runAutomaton);
+          visitor.consumeTermsMatching(parent, field, () -> runAutomaton);
           break;
         case NONE:
           break;
         case ALL:
-          visitor.consumeTermsMatching(parent, field, new ByteRunAutomaton(Automata.makeAnyString()));
+          visitor.consumeTermsMatching(parent, field, () -> new ByteRunAutomaton(Automata.makeAnyString()));
           break;
         case SINGLE:
           visitor.consumeTerms(parent, new Term(field, term));
diff --git a/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java b/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java
index 9633ea4..fa525be 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java
@@ -23,6 +23,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
 
 import com.carrotsearch.randomizedtesting.generators.RandomStrings;
 import org.apache.lucene.document.Document;
@@ -44,6 +45,7 @@ import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.RamUsageTester;
 import org.apache.lucene.util.TestUtil;
+import org.apache.lucene.util.automaton.ByteRunAutomaton;
 
 public class TermInSetQueryTest extends LuceneTestCase {
 
@@ -297,4 +299,44 @@ public class TermInSetQueryTest extends LuceneTestCase {
     // cached after two uses
     assertTrue(policy.shouldCache(query));
   }
+
+  public void testVisitor() {
+    // singleton reports back to consumeTerms()
+    TermInSetQuery singleton = new TermInSetQuery("field", new BytesRef("term1"));
+    singleton.visit(new QueryVisitor() {
+      @Override
+      public void consumeTerms(Query query, Term... terms) {
+        assertEquals(1, terms.length);
+        assertEquals(new Term("field", new BytesRef("term1")), terms[0]);
+      }
+
+      @Override
+      public void consumeTermsMatching(Query query, String field, Supplier<ByteRunAutomaton> automaton) {
+        fail("Singleton TermInSetQuery should not try to build ByteRunAutomaton");
+      }
+    });
+
+    // multiple values built into automaton
+    List<BytesRef> terms = new ArrayList<>();
+    for (int i = 0; i < 100; i++) {
+      terms.add(new BytesRef("term" + i));
+    }
+    TermInSetQuery t = new TermInSetQuery("field", terms);
+    t.visit(new QueryVisitor() {
+      @Override
+      public void consumeTerms(Query query, Term... terms) {
+        fail("TermInSetQuery with multiple terms should build automaton");
+      }
+
+      @Override
+      public void consumeTermsMatching(Query query, String field, Supplier<ByteRunAutomaton> automaton) {
+        ByteRunAutomaton a = automaton.get();
+        BytesRef test = new BytesRef("nonmatching");
+        assertFalse(a.run(test.bytes, test.offset, test.length));
+        for (BytesRef term : terms) {
+          assertTrue(a.run(term.bytes, term.offset, term.length));
+        }
+      }
+    });
+  }
 }
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/MultiTermHighlighting.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/MultiTermHighlighting.java
index ba8e85e..02bd672 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/MultiTermHighlighting.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/MultiTermHighlighting.java
@@ -19,6 +19,7 @@ package org.apache.lucene.search.uhighlight;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.function.Predicate;
+import java.util.function.Supplier;
 
 import org.apache.lucene.search.AutomatonQuery;
 import org.apache.lucene.search.BooleanClause;
@@ -80,8 +81,8 @@ final class MultiTermHighlighting {
     }
 
     @Override
-    public void consumeTermsMatching(Query query, String field, ByteRunAutomaton automaton) {
-      runAutomata.add(LabelledCharArrayMatcher.wrap(query.toString(), automaton));
+    public void consumeTermsMatching(Query query, String field, Supplier<ByteRunAutomaton> automaton) {
+      runAutomata.add(LabelledCharArrayMatcher.wrap(query.toString(), automaton.get()));
     }
 
   }