You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sh...@apache.org on 2017/06/21 07:10:32 UTC

[06/50] [abbrv] lucene-solr:feature/autoscaling: LUCENE-7719: Generalize UnifiedHighlighter's support for AutomatonQuery

LUCENE-7719: Generalize UnifiedHighlighter's support for AutomatonQuery


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

Branch: refs/heads/feature/autoscaling
Commit: d0b9d3459fd097dba677cdda170632f6fca5e042
Parents: b49ce68
Author: David Smiley <ds...@apache.org>
Authored: Fri Jun 16 22:57:51 2017 -0400
Committer: David Smiley <ds...@apache.org>
Committed: Fri Jun 16 22:57:51 2017 -0400

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   3 +
 .../apache/lucene/search/AutomatonQuery.java    |   7 ++
 .../uhighlight/MemoryIndexOffsetStrategy.java   |   3 +-
 .../uhighlight/MultiTermHighlighting.java       | 124 ++++++++-----------
 .../uhighlight/TestUnifiedHighlighterMTQ.java   |  69 ++++++++++-
 5 files changed, 128 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d0b9d345/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 595ed12..e708888 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -130,6 +130,9 @@ Other
 * LUCENE-7852: Correct copyright year(s) in lucene/LICENSE.txt file.
   (Christine Poerschke, Steve Rowe)
 
+* LUCENE-7719: Generalized the UnifiedHighlighter's support for AutomatonQuery
+  for character & binary automata. Added AutomatonQuery.isBinary. (David Smiley)
+
 ======================= Lucene 6.7.0 =======================
 
 New Features

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d0b9d345/lucene/core/src/java/org/apache/lucene/search/AutomatonQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/AutomatonQuery.java b/lucene/core/src/java/org/apache/lucene/search/AutomatonQuery.java
index 3b113a2..7fb155d 100644
--- a/lucene/core/src/java/org/apache/lucene/search/AutomatonQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/AutomatonQuery.java
@@ -51,6 +51,7 @@ public class AutomatonQuery extends MultiTermQuery {
   protected final CompiledAutomaton compiled;
   /** term containing the field, and possibly some pattern structure */
   protected final Term term;
+  protected final boolean automatonIsBinary;
 
   /**
    * Create a new AutomatonQuery from an {@link Automaton}.
@@ -98,6 +99,7 @@ public class AutomatonQuery extends MultiTermQuery {
     super(term.field());
     this.term = term;
     this.automaton = automaton;
+    this.automatonIsBinary = isBinary;
     // TODO: we could take isFinite too, to save a bit of CPU in CompiledAutomaton ctor?:
     this.compiled = new CompiledAutomaton(automaton, null, true, maxDeterminizedStates, isBinary);
   }
@@ -154,4 +156,9 @@ public class AutomatonQuery extends MultiTermQuery {
   public Automaton getAutomaton() {
     return automaton;
   }
+
+  /** Is this a binary (byte) oriented automaton. See the constructor.  */
+  public boolean isAutomatonBinary() {
+    return automatonIsBinary;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d0b9d345/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/MemoryIndexOffsetStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/MemoryIndexOffsetStrategy.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/MemoryIndexOffsetStrategy.java
index 0001a80..cf7a3c9 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/MemoryIndexOffsetStrategy.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/MemoryIndexOffsetStrategy.java
@@ -83,8 +83,7 @@ public class MemoryIndexOffsetStrategy extends AnalysisOffsetStrategy {
       return allAutomata.get(0);
     }
     //TODO it'd be nice if we could get at the underlying Automaton in CharacterRunAutomaton so that we
-    //  could union them all. But it's not exposed, and note TermRangeQuery isn't modelled as an Automaton
-    //  by MultiTermHighlighting.
+    //  could union them all. But it's not exposed, and sometimes the automaton is byte (not char) oriented
 
     // Return an aggregate CharacterRunAutomaton of others
     return new CharacterRunAutomaton(Automata.makeEmpty()) {// the makeEmpty() is bogus; won't be used

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d0b9d345/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/MultiTermHighlighting.java
----------------------------------------------------------------------
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 15f4bdd..e745d78 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,12 +19,10 @@ package org.apache.lucene.search.uhighlight;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Comparator;
 import java.util.List;
 import java.util.function.Function;
 import java.util.function.Predicate;
 
-import org.apache.lucene.index.Term;
 import org.apache.lucene.search.AutomatonQuery;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
@@ -32,19 +30,17 @@ import org.apache.lucene.search.BoostQuery;
 import org.apache.lucene.search.ConstantScoreQuery;
 import org.apache.lucene.search.DisjunctionMaxQuery;
 import org.apache.lucene.search.FuzzyQuery;
-import org.apache.lucene.search.PrefixQuery;
 import org.apache.lucene.search.Query;
-import org.apache.lucene.search.TermRangeQuery;
 import org.apache.lucene.search.spans.SpanBoostQuery;
 import org.apache.lucene.search.spans.SpanMultiTermQueryWrapper;
 import org.apache.lucene.search.spans.SpanNearQuery;
 import org.apache.lucene.search.spans.SpanNotQuery;
 import org.apache.lucene.search.spans.SpanOrQuery;
 import org.apache.lucene.search.spans.SpanPositionCheckQuery;
-import org.apache.lucene.util.CharsRef;
 import org.apache.lucene.util.UnicodeUtil;
 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.CharacterRunAutomaton;
 import org.apache.lucene.util.automaton.LevenshteinAutomata;
 import org.apache.lucene.util.automaton.Operations;
@@ -110,18 +106,6 @@ class MultiTermHighlighting {
     } else if (lookInSpan && query instanceof SpanMultiTermQueryWrapper) {
       list.addAll(Arrays.asList(extractAutomata(((SpanMultiTermQueryWrapper<?>) query).getWrappedQuery(),
           fieldMatcher, lookInSpan, preRewriteFunc)));
-    } else if (query instanceof PrefixQuery) {
-      final PrefixQuery pq = (PrefixQuery) query;
-      Term prefix = pq.getPrefix();
-      if (fieldMatcher.test(prefix.field())) {
-        list.add(new CharacterRunAutomaton(Operations.concatenate(Automata.makeString(prefix.text()),
-            Automata.makeAnyString())) {
-          @Override
-          public String toString() {
-            return pq.toString();
-          }
-        });
-      }
     } else if (query instanceof FuzzyQuery) {
       final FuzzyQuery fq = (FuzzyQuery) query;
       if (fieldMatcher.test(fq.getField())) {
@@ -143,69 +127,63 @@ class MultiTermHighlighting {
           }
         });
       }
-    } else if (query instanceof TermRangeQuery) {
-      final TermRangeQuery tq = (TermRangeQuery) query;
-      if (fieldMatcher.test(tq.getField())) {
-        final CharsRef lowerBound;
-        if (tq.getLowerTerm() == null) {
-          lowerBound = null;
-        } else {
-          lowerBound = new CharsRef(tq.getLowerTerm().utf8ToString());
-        }
-
-        final CharsRef upperBound;
-        if (tq.getUpperTerm() == null) {
-          upperBound = null;
-        } else {
-          upperBound = new CharsRef(tq.getUpperTerm().utf8ToString());
-        }
-
-        final boolean includeLower = tq.includesLower();
-        final boolean includeUpper = tq.includesUpper();
-        final CharsRef scratch = new CharsRef();
-
-        @SuppressWarnings("deprecation")
-        final Comparator<CharsRef> comparator = CharsRef.getUTF16SortedAsUTF8Comparator();
+    } else if (query instanceof AutomatonQuery) {
+      final AutomatonQuery aq = (AutomatonQuery) query;
+      if (fieldMatcher.test(aq.getField())) {
 
-        // this is *not* an automaton, but its very simple
-        list.add(new CharacterRunAutomaton(Automata.makeEmpty()) {
-          @Override
-          public boolean run(char[] s, int offset, int length) {
-            scratch.chars = s;
-            scratch.offset = offset;
-            scratch.length = length;
+        if (aq.isAutomatonBinary() == false) { // note: is the case for WildcardQuery, RegexpQuery
+          list.add(new CharacterRunAutomaton(aq.getAutomaton()) {
+            @Override
+            public String toString() {
+              return aq.toString();
+            }
+          });
+        } else { // note: is the case for PrefixQuery, TermRangeQuery
+          // byte oriented automaton:
+          list.add(new CharacterRunAutomaton(Automata.makeEmpty()) { // empty here is bogus just to satisfy API
+            //   TODO can we get access to the aq.compiledAutomaton.runAutomaton ?
+            ByteRunAutomaton byteRunAutomaton =
+                new ByteRunAutomaton(aq.getAutomaton(), true, Operations.DEFAULT_MAX_DETERMINIZED_STATES);
 
-            if (lowerBound != null) {
-              int cmp = comparator.compare(scratch, lowerBound);
-              if (cmp < 0 || (!includeLower && cmp == 0)) {
-                return false;
+            @Override
+            public boolean run(char[] chars, int offset, int length) {
+              int state = 0;
+              final int maxIdx = offset + length;
+              for (int i = offset; i < maxIdx; i++) {
+                final int code = chars[i];
+                int b;
+                // UTF16 to UTF8   (inlined logic from UnicodeUtil.UTF16toUTF8 )
+                if (code < 0x80) {
+                  state = byteRunAutomaton.step(state, code);
+                  if (state == -1) return false;
+                } else if (code < 0x800) {
+                  b = (0xC0 | (code >> 6));
+                  state = byteRunAutomaton.step(state, b);
+                  if (state == -1) return false;
+                  b = (0x80 | (code & 0x3F));
+                  state = byteRunAutomaton.step(state, b);
+                  if (state == -1) return false;
+                } else {
+                  // more complex
+                  byte[] utf8Bytes = new byte[4 * (maxIdx - i)];
+                  int utf8Len = UnicodeUtil.UTF16toUTF8(chars, i, maxIdx - i, utf8Bytes);
+                  for (int utfIdx = 0; utfIdx < utf8Len; utfIdx++) {
+                    state = byteRunAutomaton.step(state, utf8Bytes[utfIdx] & 0xFF);
+                    if (state == -1) return false;
+                  }
+                  break;
+                }
               }
+              return byteRunAutomaton.isAccept(state);
             }
 
-            if (upperBound != null) {
-              int cmp = comparator.compare(scratch, upperBound);
-              if (cmp > 0 || (!includeUpper && cmp == 0)) {
-                return false;
-              }
+            @Override
+            public String toString() {
+              return aq.toString();
             }
-            return true;
-          }
+          });
+        }
 
-          @Override
-          public String toString() {
-            return tq.toString();
-          }
-        });
-      }
-    } else if (query instanceof AutomatonQuery) {
-      final AutomatonQuery aq = (AutomatonQuery) query;
-      if (fieldMatcher.test(aq.getField())) {
-        list.add(new CharacterRunAutomaton(aq.getAutomaton()) {
-          @Override
-          public String toString() {
-            return aq.toString();
-          }
-        });
       }
     }
     return list.toArray(new CharacterRunAutomaton[list.size()]);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d0b9d345/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterMTQ.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterMTQ.java b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterMTQ.java
index 57f174f..ac99c79 100644
--- a/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterMTQ.java
+++ b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterMTQ.java
@@ -24,11 +24,13 @@ import java.util.List;
 import java.util.Objects;
 
 import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
+import com.carrotsearch.randomizedtesting.generators.RandomStrings;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.core.KeywordAnalyzer;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
@@ -63,16 +65,15 @@ import org.apache.lucene.search.spans.SpanQuery;
 import org.apache.lucene.search.spans.SpanTermQuery;
 import org.apache.lucene.search.spans.SpanWeight;
 import org.apache.lucene.store.BaseDirectoryWrapper;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
+import org.apache.lucene.util.UnicodeUtil;
 import org.junit.After;
 import org.junit.Before;
 
 /**
  * Some tests that highlight wildcard, fuzzy, etc queries.
  */
-@SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom", "Lucene3x"})
-@LuceneTestCase.SuppressSysoutChecks(bugUrl = "")//Gradle interferes with this Lucene test rule
 public class TestUnifiedHighlighterMTQ extends LuceneTestCase {
 
   final FieldType fieldType;
@@ -1079,4 +1080,66 @@ public class TestUnifiedHighlighterMTQ extends LuceneTestCase {
     assertEquals("[<b>я</b>]", Arrays.toString(snippets));
     ir.close();
   }
+
+  // LUCENE-7719
+  public void testMultiByteMTQ() throws IOException {
+    Analyzer analyzer = new KeywordAnalyzer();
+    try (RandomIndexWriter iw = new RandomIndexWriter(random(), dir, analyzer)) {
+      for (int attempt = 0; attempt < 20; attempt++) {
+        iw.deleteAll();
+        String field = "title";
+        String value = RandomStrings.randomUnicodeOfLength(random(), 3);
+        if (value.contains(UnifiedHighlighter.MULTIVAL_SEP_CHAR+"")) { // will throw things off
+          continue;
+        }
+        int[] valuePoints = value.codePoints().toArray();
+
+        iw.addDocument(Collections.singleton(
+            new Field(field, value, fieldType)));
+        iw.commit();
+        try (IndexReader ir = iw.getReader()) {
+          IndexSearcher searcher = newSearcher(ir);
+          UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, analyzer);
+          highlighter.setBreakIterator(WholeBreakIterator::new);
+
+          // Test PrefixQuery
+          Query query = new PrefixQuery(new Term(field,
+              UnicodeUtil.newString(valuePoints, 0, 1)));
+          highlightAndAssertMatch(searcher, highlighter, query, field, value);
+
+          // Test TermRangeQuery
+          query = new TermRangeQuery(field,
+              new BytesRef(value),
+              new BytesRef(value),
+              true, true );
+          highlightAndAssertMatch(searcher, highlighter, query, field, value);
+
+          // Test FuzzyQuery
+          query = new FuzzyQuery(new Term(field, value + "Z"), 1);
+          highlightAndAssertMatch(searcher, highlighter, query, field, value);
+
+          if (valuePoints.length != 3) {
+            continue; // even though we ask RandomStrings for a String with 3 code points, it seems sometimes it's less
+          }
+
+          // Test WildcardQuery
+          query = new WildcardQuery(new Term(field,
+              new StringBuilder()
+                  .append(WildcardQuery.WILDCARD_ESCAPE).appendCodePoint(valuePoints[0])
+                  .append(WildcardQuery.WILDCARD_CHAR)
+                  .append(WildcardQuery.WILDCARD_ESCAPE).appendCodePoint(valuePoints[2]).toString()));
+          highlightAndAssertMatch(searcher, highlighter, query, field, value);
+
+          //TODO hmmm; how to randomly generate RegexpQuery? Low priority; we've covered the others well.
+        }
+      }
+    }
+  }
+
+  private void highlightAndAssertMatch(IndexSearcher searcher, UnifiedHighlighter highlighter, Query query, String field, String fieldVal) throws IOException {
+    TopDocs topDocs = searcher.search(query, 1);
+    assertEquals(1, topDocs.totalHits);
+    String[] snippets = highlighter.highlight(field, query, topDocs);
+    assertEquals("[<b>"+fieldVal+"</b>]", Arrays.toString(snippets));
+  }
 }