You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ds...@apache.org on 2019/06/21 21:12:56 UTC

[lucene-solr] branch branch_8x updated: LUCENE-8848 LUCENE-7757 LUCENE-8492: UnifiedHighlighter.hasUnrecognizedQuery The UH now detects that parts of the query are not understood by it. When found, it highlights more safely/reliably. Fixes compatibility with complex and surround query parsers.

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

dsmiley 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 df4ebaf  LUCENE-8848 LUCENE-7757 LUCENE-8492: UnifiedHighlighter.hasUnrecognizedQuery The UH now detects that parts of the query are not understood by it. When found, it highlights more safely/reliably. Fixes compatibility with complex and surround query parsers.
df4ebaf is described below

commit df4ebaf030cdc5aeeba527aab23febcb6dfc9601
Author: David Smiley <ds...@apache.org>
AuthorDate: Fri Jun 21 17:05:56 2019 -0400

    LUCENE-8848 LUCENE-7757 LUCENE-8492: UnifiedHighlighter.hasUnrecognizedQuery
    The UH now detects that parts of the query are not understood by it.
    When found, it highlights more safely/reliably.
    Fixes compatibility with complex and surround query parsers.
    
    (cherry picked from commit 54cc70127b22083198f1c44f83ccf4cdf769ac77)
---
 lucene/CHANGES.txt                                 |   5 +
 .../uhighlight/MemoryIndexOffsetStrategy.java      |   9 +-
 .../search/uhighlight/MultiTermHighlighting.java   |   8 ++
 .../search/uhighlight/NoOpOffsetStrategy.java      |   2 +-
 .../uhighlight/TokenStreamOffsetStrategy.java      |  16 +---
 .../lucene/search/uhighlight/UHComponents.java     |   8 +-
 .../search/uhighlight/UnifiedHighlighter.java      |  66 +++++++++----
 .../search/uhighlight/TestUnifiedHighlighter.java  | 103 +++++++++++++++++++++
 .../TestUnifiedHighlighterExtensibility.java       |  33 ++++---
 .../solr/highlight/TestUnifiedSolrHighlighter.java |  12 +++
 10 files changed, 218 insertions(+), 44 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 9d944f3..0a24caa 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -85,6 +85,11 @@ Improvements
 * LUCENE-8845: Allow Intervals.prefix() and Intervals.wildcard() to specify
   their maximum allowed expansions (Alan Woodward)
 
+* LUCENE-8848 LUCENE-7757 LUCENE-8492: The UnifiedHighlighter now detects that parts of the query are not understood by
+  it, and thus it should not make optimizations that result in no highlights or slow highlighting.  This generally works
+  best for WEIGHT_MATCHES mode.  Consequently queries produced by ComplexPhraseQueryParser and the surround QueryParser
+  will now highlight correctly. (David Smiley)
+
 Optimizations
 
 * LUCENE-8796: Use exponential search instead of binary search in
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 1f4455f..e53d6e4 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
@@ -57,6 +57,11 @@ public class MemoryIndexOffsetStrategy extends AnalysisOffsetStrategy {
    * Build one {@link CharacterRunAutomaton} matching any term the query might match.
    */
   private static CharacterRunAutomaton buildCombinedAutomaton(UHComponents components) {
+    // We don't know enough about the query to do this confidently
+    if (components.getTerms() == null || components.getAutomata() == null) {
+      return null;
+    }
+
     List<CharacterRunAutomaton> allAutomata = new ArrayList<>();
     if (components.getTerms().length > 0) {
       allAutomata.add(new CharacterRunAutomaton(Automata.makeStringUnion(Arrays.asList(components.getTerms()))));
@@ -93,7 +98,9 @@ public class MemoryIndexOffsetStrategy extends AnalysisOffsetStrategy {
     TokenStream tokenStream = tokenStream(content);
 
     // Filter the tokenStream to applicable terms
-    tokenStream = newKeepWordFilter(tokenStream, preMemIndexFilterAutomaton);
+    if (preMemIndexFilterAutomaton != null) {
+      tokenStream = newKeepWordFilter(tokenStream, preMemIndexFilterAutomaton);
+    }
     memoryIndex.reset();
     memoryIndex.addField(getField(), tokenStream);//note: calls tokenStream.reset() & close()
 
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 d079599..8181c26 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
@@ -53,6 +53,14 @@ final class MultiTermHighlighting {
     return collector.runAutomata.toArray(new CharacterRunAutomaton[0]);
   }
 
+  /**
+   * Indicates if the the leaf query (from {@link QueryVisitor#visitLeaf(Query)}) is a type of query that
+   * we can extract automata from.
+   */
+  public static boolean canExtractAutomataFromLeafQuery(Query query) {
+    return query instanceof AutomatonQuery || query instanceof FuzzyQuery;
+  }
+
   private static class AutomataCollector extends QueryVisitor {
 
     List<CharacterRunAutomaton> runAutomata = new ArrayList<>();
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/NoOpOffsetStrategy.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/NoOpOffsetStrategy.java
index 80528ce..08f2b12 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/NoOpOffsetStrategy.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/NoOpOffsetStrategy.java
@@ -34,7 +34,7 @@ public class NoOpOffsetStrategy extends FieldOffsetStrategy {
   public static final NoOpOffsetStrategy INSTANCE = new NoOpOffsetStrategy();
 
   private NoOpOffsetStrategy() {
-    super(new UHComponents("_ignored_", (s) -> false, new MatchNoDocsQuery(), new BytesRef[0], PhraseHelper.NONE, new CharacterRunAutomaton[0], Collections.emptySet()));
+    super(new UHComponents("_ignored_", (s) -> false, new MatchNoDocsQuery(), new BytesRef[0], PhraseHelper.NONE, new CharacterRunAutomaton[0], false, Collections.emptySet()));
   }
 
   @Override
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/TokenStreamOffsetStrategy.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/TokenStreamOffsetStrategy.java
index 677ee4b..a7282b6 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/TokenStreamOffsetStrategy.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/TokenStreamOffsetStrategy.java
@@ -34,21 +34,15 @@ import org.apache.lucene.util.automaton.CharacterRunAutomaton;
  */
 public class TokenStreamOffsetStrategy extends AnalysisOffsetStrategy {
 
-  private static final BytesRef[] ZERO_LEN_BYTES_REF_ARRAY = new BytesRef[0];
+  private final CharacterRunAutomaton[] combinedAutomata;
 
   public TokenStreamOffsetStrategy(UHComponents components, Analyzer indexAnalyzer) {
-    super(new UHComponents(
-            components.getField(),
-            components.getFieldMatcher(),
-            components.getQuery(),
-            ZERO_LEN_BYTES_REF_ARRAY,
-            components.getPhraseHelper(),
-            convertTermsToAutomata(components.getTerms(), components.getAutomata()),
-            components.getHighlightFlags()),
-        indexAnalyzer);
+    super(components, indexAnalyzer);
     assert components.getPhraseHelper().hasPositionSensitivity() == false;
+    combinedAutomata = convertTermsToAutomata(components.getTerms(), components.getAutomata());
   }
 
+  //TODO this is inefficient; instead build a union automata just for terms part.
   private static CharacterRunAutomaton[] convertTermsToAutomata(BytesRef[] terms, CharacterRunAutomaton[] automata) {
     CharacterRunAutomaton[] newAutomata = new CharacterRunAutomaton[terms.length + automata.length];
     for (int i = 0; i < terms.length; i++) {
@@ -67,7 +61,7 @@ public class TokenStreamOffsetStrategy extends AnalysisOffsetStrategy {
 
   @Override
   public OffsetsEnum getOffsetsEnum(LeafReader reader, int docId, String content) throws IOException {
-    return new TokenStreamOffsetsEnum(tokenStream(content), components.getAutomata());
+    return new TokenStreamOffsetsEnum(tokenStream(content), combinedAutomata);
   }
 
   private static class TokenStreamOffsetsEnum extends OffsetsEnum {
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/UHComponents.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/UHComponents.java
index eed1e9c..4af6d70 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/UHComponents.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/UHComponents.java
@@ -36,17 +36,19 @@ public class UHComponents {
   private final BytesRef[] terms; // Query: all terms we extracted (some may be position sensitive)
   private final PhraseHelper phraseHelper; // Query: position-sensitive information
   private final CharacterRunAutomaton[] automata; // Query: wildcards (i.e. multi-term query), not position sensitive
+  private final boolean hasUnrecognizedQueryPart; // Query: if part of the query (other than the extracted terms / automata) is a leaf we don't know
   private final Set<UnifiedHighlighter.HighlightFlag> highlightFlags;
 
   public UHComponents(String field, Predicate<String> fieldMatcher, Query query,
                       BytesRef[] terms, PhraseHelper phraseHelper, CharacterRunAutomaton[] automata,
-                      Set<UnifiedHighlighter.HighlightFlag> highlightFlags) {
+                      boolean hasUnrecognizedQueryPart, Set<UnifiedHighlighter.HighlightFlag> highlightFlags) {
     this.field = field;
     this.fieldMatcher = fieldMatcher;
     this.query = query;
     this.terms = terms;
     this.phraseHelper = phraseHelper;
     this.automata = automata;
+    this.hasUnrecognizedQueryPart = hasUnrecognizedQueryPart;
     this.highlightFlags = highlightFlags;
   }
 
@@ -74,6 +76,10 @@ public class UHComponents {
     return automata;
   }
 
+  public boolean hasUnrecognizedQueryPart() {
+    return hasUnrecognizedQueryPart;
+  }
+
   public Set<UnifiedHighlighter.HighlightFlag> getHighlightFlags() {
     return highlightFlags;
   }
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/UnifiedHighlighter.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/UnifiedHighlighter.java
index f4b1ab1..59ca7d3 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/UnifiedHighlighter.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/UnifiedHighlighter.java
@@ -52,6 +52,8 @@ import org.apache.lucene.index.StoredFieldVisitor;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.MatchNoDocsQuery;
 import org.apache.lucene.search.MultiTermQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.QueryVisitor;
@@ -751,13 +753,8 @@ public class UnifiedHighlighter {
   }
 
   protected FieldHighlighter getFieldHighlighter(String field, Query query, Set<Term> allTerms, int maxPassages) {
-    Predicate<String> fieldMatcher = getFieldMatcher(field);
-    BytesRef[] terms = filterExtractedTerms(fieldMatcher, allTerms);
-    Set<HighlightFlag> highlightFlags = getFlags(field);
-    PhraseHelper phraseHelper = getPhraseHelper(field, query, highlightFlags);
-    CharacterRunAutomaton[] automata = getAutomata(field, query, highlightFlags);
-    OffsetSource offsetSource = getOptimizedOffsetSource(field, terms, phraseHelper, automata);
-    UHComponents components = new UHComponents(field, fieldMatcher, query, terms, phraseHelper, automata, highlightFlags);
+    UHComponents components = getHighlightComponents(field, query, allTerms);
+    OffsetSource offsetSource = getOptimizedOffsetSource(components);
     return new FieldHighlighter(field,
         getOffsetStrategy(offsetSource, components),
         new SplittingBreakIterator(getBreakIterator(field), UnifiedHighlighter.MULTIVAL_SEP_CHAR),
@@ -767,6 +764,41 @@ public class UnifiedHighlighter {
         getFormatter(field));
   }
 
+  protected UHComponents getHighlightComponents(String field, Query query, Set<Term> allTerms) {
+    Predicate<String> fieldMatcher = getFieldMatcher(field);
+    Set<HighlightFlag> highlightFlags = getFlags(field);
+    PhraseHelper phraseHelper = getPhraseHelper(field, query, highlightFlags);
+    boolean queryHasUnrecognizedPart = hasUnrecognizedQuery(fieldMatcher, query);
+    BytesRef[] terms = null;
+    CharacterRunAutomaton[] automata = null;
+    if (!highlightFlags.contains(HighlightFlag.WEIGHT_MATCHES) || !queryHasUnrecognizedPart) {
+      terms = filterExtractedTerms(fieldMatcher, allTerms);
+      automata = getAutomata(field, query, highlightFlags);
+    } // otherwise don't need to extract
+    return new UHComponents(field, fieldMatcher, query, terms, phraseHelper, automata, queryHasUnrecognizedPart, highlightFlags);
+  }
+
+  protected boolean hasUnrecognizedQuery(Predicate<String> fieldMatcher, Query query) {
+    boolean[] hasUnknownLeaf = new boolean[1];
+    query.visit(new QueryVisitor() {
+      @Override
+      public boolean acceptField(String field) {
+        // checking hasUnknownLeaf is a trick to exit early
+        return hasUnknownLeaf[0] == false && fieldMatcher.test(field);
+      }
+
+      @Override
+      public void visitLeaf(Query query) {
+        if (MultiTermHighlighting.canExtractAutomataFromLeafQuery(query) == false) {
+          if (!(query instanceof MatchAllDocsQuery || query instanceof MatchNoDocsQuery)) {
+            hasUnknownLeaf[0] = true;
+          }
+        }
+      }
+    });
+    return hasUnknownLeaf[0];
+  }
+
   protected static BytesRef[] filterExtractedTerms(Predicate<String> fieldMatcher, Set<Term> queryTerms) {
     // Strip off the redundant field and sort the remaining terms
     SortedSet<BytesRef> filteredTerms = new TreeSet<>();
@@ -820,26 +852,26 @@ public class UnifiedHighlighter {
         : ZERO_LEN_AUTOMATA_ARRAY;
   }
 
-  protected OffsetSource getOptimizedOffsetSource(String field, BytesRef[] terms, PhraseHelper phraseHelper, CharacterRunAutomaton[] automata) {
-    OffsetSource offsetSource = getOffsetSource(field);
+  protected OffsetSource getOptimizedOffsetSource(UHComponents components) {
+    OffsetSource offsetSource = getOffsetSource(components.getField());
 
-    if (terms.length == 0 && automata.length == 0 && !phraseHelper.willRewrite()) {
+    // null automata means unknown, so assume a possibility
+    boolean mtqOrRewrite = components.getAutomata() == null || components.getAutomata().length > 0
+        || components.getPhraseHelper().willRewrite() || components.hasUnrecognizedQueryPart();
+
+    // null terms means unknown, so assume something to highlight
+    if (mtqOrRewrite == false && components.getTerms() != null && components.getTerms().length == 0) {
       return OffsetSource.NONE_NEEDED; //nothing to highlight
     }
 
     switch (offsetSource) {
       case POSTINGS:
-        if (phraseHelper.willRewrite()) {
-          // We can't choose the postings offset source when there is "rewriting" in the strict phrase
-          // processing (rare but possible). Postings requires knowing all the terms (except wildcards)
-          // up front.
-          return OffsetSource.ANALYSIS;
-        } else if (automata.length > 0) {
+        if (mtqOrRewrite) { // may need to see scan through all terms for the highlighted document efficiently
           return OffsetSource.ANALYSIS;
         }
         break;
       case POSTINGS_WITH_TERM_VECTORS:
-        if (!phraseHelper.willRewrite() && automata.length == 0) {
+        if (mtqOrRewrite == false) {
           return OffsetSource.POSTINGS; //We don't need term vectors
         }
         break;
diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighter.java b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighter.java
index 3e2cc2e..2565e7e 100644
--- a/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighter.java
+++ b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighter.java
@@ -48,10 +48,13 @@ import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.PhraseQuery;
 import org.apache.lucene.search.PrefixQuery;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryVisitor;
 import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.Weight;
 import org.apache.lucene.search.uhighlight.UnifiedHighlighter.HighlightFlag;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
@@ -1357,4 +1360,104 @@ public class TestUnifiedHighlighter extends LuceneTestCase {
 
     ir.close();
   }
+
+  public void testNotReanalyzed() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    if (fieldType == UHTestHelper.reanalysisType) {
+      return; // we're testing the *other* cases
+    }
+
+    Field body = new Field("body", "", fieldType);
+    Document doc = new Document();
+    doc.add(body);
+
+    body.setStringValue("This is a test. Just a test highlighting from postings. Feel free to ignore.");
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, new Analyzer() {
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName) {
+        throw new AssertionError("shouldn't be called");
+      }
+    });
+    Query query = new TermQuery(new Term("body", "highlighting"));
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(1, topDocs.totalHits.value);
+    String snippets[] = highlighter.highlight("body", query, topDocs);
+    assertEquals(1, snippets.length);
+    assertEquals("Just a test <b>highlighting</b> from postings. ", snippets[0]);
+
+    ir.close();
+  }
+
+  public void testUnknownQueryWithWeightMatches() throws IOException {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Field body = new Field("body", "", fieldType);
+    Document doc = new Document();
+    doc.add(body);
+
+    body.setStringValue("Test a one sentence document.");
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, indexAnalyzer,
+        EnumSet.of(HighlightFlag.WEIGHT_MATCHES), null);
+    Query query = new BooleanQuery.Builder()
+        // simple term query body:one
+        .add(new TermQuery(new Term(body.name(), "one")), BooleanClause.Occur.MUST)
+        // a custom query, a leaf, that which matches body:sentence
+        //    Note this isn't even an MTQ.  What matters is that Weight.matches works.
+        .add(new Query() {
+          @Override
+          public String toString(String field) {
+            return "bogus";
+          }
+
+          @Override
+          public Query rewrite(IndexReader reader) {
+            return this;
+          }
+
+          // we don't visit terms, and we don't expose an automata.  Thus this appears as some unknown leaf.
+          @Override
+          public void visit(QueryVisitor visitor) {
+            if (visitor.acceptField(body.name())) {
+              visitor.visitLeaf(this);
+            }
+          }
+
+          @Override
+          public boolean equals(Object obj) {
+            return this == obj;
+          }
+
+          @Override
+          public int hashCode() {
+            return System.identityHashCode(this);
+          }
+
+          @Override
+          public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
+            //TODO maybe should loop through index terms to show we can see other terms
+            return new TermQuery(new Term(body.name(), "sentence")).createWeight(searcher, scoreMode, boost);
+          }
+        }, BooleanClause.Occur.MUST)
+        .build();
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(1, topDocs.totalHits.value);
+    String[] snippets = highlighter.highlight("body", query, topDocs);
+    assertEquals(1, snippets.length);
+    assertEquals("Test a <b>one</b> <b>sentence</b> document.", snippets[0]);
+
+    ir.close();
+  }
 }
diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/visibility/TestUnifiedHighlighterExtensibility.java b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/visibility/TestUnifiedHighlighterExtensibility.java
index 3910b5a..def4431 100644
--- a/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/visibility/TestUnifiedHighlighterExtensibility.java
+++ b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/visibility/TestUnifiedHighlighterExtensibility.java
@@ -65,7 +65,7 @@ public class TestUnifiedHighlighterExtensibility extends LuceneTestCase {
         (s) -> false,
         new MatchAllDocsQuery(), new BytesRef[0],
         PhraseHelper.NONE,
-        new CharacterRunAutomaton[0], Collections.emptySet())) {
+        new CharacterRunAutomaton[0], false, Collections.emptySet())) {
       @Override
       public UnifiedHighlighter.OffsetSource getOffsetSource() {
         return offsetSource;
@@ -152,22 +152,18 @@ public class TestUnifiedHighlighterExtensibility extends LuceneTestCase {
       @Override
       protected FieldHighlighter getFieldHighlighter(String field, Query query, Set<Term> allTerms, int maxPassages) {
         // THIS IS A COPY of the superclass impl; but use CustomFieldHighlighter
-        Predicate<String> fieldMatcher = getFieldMatcher(field);
-        BytesRef[] terms = filterExtractedTerms(fieldMatcher, allTerms);
-        Set<HighlightFlag> highlightFlags = getFlags(field);
-        PhraseHelper phraseHelper = getPhraseHelper(field, query, highlightFlags);
-        CharacterRunAutomaton[] automata = getAutomata(field, query, highlightFlags);
-        OffsetSource offsetSource = getOptimizedOffsetSource(field, terms, phraseHelper, automata);
+        UHComponents components = getHighlightComponents(field, query, allTerms);
+        OffsetSource offsetSource = getOptimizedOffsetSource(components);
 
-        UHComponents components = new UHComponents(field, fieldMatcher, query, terms, phraseHelper, automata, highlightFlags);
         // test all is accessible
-        components.getAutomata();
-        components.getPhraseHelper();
-        components.getTerms();
         components.getField();
-        components.getHighlightFlags();
-        components.getQuery();
         components.getFieldMatcher();
+        components.getQuery();
+        components.getTerms();
+        components.getPhraseHelper();
+        components.getAutomata();
+        components.hasUnrecognizedQueryPart();
+        components.getHighlightFlags();
 
         return new CustomFieldHighlighter(field,
             getOffsetStrategy(offsetSource, components),
@@ -179,6 +175,17 @@ public class TestUnifiedHighlighterExtensibility extends LuceneTestCase {
       }
 
       @Override
+      protected UHComponents getHighlightComponents(String field, Query query, Set<Term> allTerms) {
+        Predicate<String> fieldMatcher = getFieldMatcher(field);
+        BytesRef[] terms = filterExtractedTerms(fieldMatcher, allTerms);
+        Set<HighlightFlag> highlightFlags = getFlags(field);
+        PhraseHelper phraseHelper = getPhraseHelper(field, query, highlightFlags);
+        CharacterRunAutomaton[] automata = getAutomata(field, query, highlightFlags);
+        boolean queryHasUnrecognizedPart = false;
+        return new UHComponents(field, fieldMatcher, query, terms, phraseHelper, automata, queryHasUnrecognizedPart, highlightFlags);
+      }
+
+      @Override
       protected FieldOffsetStrategy getOffsetStrategy(OffsetSource offsetSource, UHComponents components) {
         return super.getOffsetStrategy(offsetSource, components);
       }
diff --git a/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java b/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java
index 2a3e3a7..f03376d 100644
--- a/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java
+++ b/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java
@@ -302,4 +302,16 @@ public class TestUnifiedSolrHighlighter extends SolrTestCaseJ4 {
         "//lst[@name='highlighting']/lst[@name='101']/arr/str[1]='<em>alpha</em> <em>bravo</em> charlie'");
   }
 
+  // LUCENE-8492
+  public void testSurroundQParser() {
+    assertQ(req("q", "{!surround df=text}2w(second, document)", "hl", "true", "hl.fl", "text"),
+        "count(//lst[@name='highlighting']/lst[@name='102']/arr[@name='text']/*)=1");
+  }
+
+  // LUCENE-7757
+  public void testComplexPhraseQParser() {
+    assertQ(req("q", "{!complexphrase df=text}(\"sec* doc*\")", "hl", "true", "hl.fl", "text"),
+        "count(//lst[@name='highlighting']/lst[@name='102']/arr[@name='text']/*)=1");
+  }
+
 }