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 2018/08/30 03:30:16 UTC

[2/2] lucene-solr:master: LUCENE-8286: UnifiedHighlighter: new HighlightFlag.WEIGHT_MATCHES for MatchesIterator API. Other API changes: New UHComponents, and FieldOffsetStrategy takes a LeafReader not IndexReader now. Closes #409

LUCENE-8286: UnifiedHighlighter: new HighlightFlag.WEIGHT_MATCHES for MatchesIterator API.
Other API changes: New UHComponents, and FieldOffsetStrategy takes a LeafReader not IndexReader now.
Closes #409


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

Branch: refs/heads/master
Commit: b19ae942f154924b9108c4e0409865128f2a07d4
Parents: 18874a6
Author: David Smiley <ds...@apache.org>
Authored: Wed Aug 29 23:29:25 2018 -0400
Committer: David Smiley <ds...@apache.org>
Committed: Wed Aug 29 23:29:25 2018 -0400

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   7 +
 .../uhighlight/AnalysisOffsetStrategy.java      |  16 +-
 .../search/uhighlight/FieldHighlighter.java     |   5 +-
 .../search/uhighlight/FieldOffsetStrategy.java  | 121 ++++++++---
 .../uhighlight/MemoryIndexOffsetStrategy.java   |  51 ++---
 .../search/uhighlight/NoOpOffsetStrategy.java   |   8 +-
 .../lucene/search/uhighlight/OffsetsEnum.java   | 216 ++++++++++++++++++-
 .../OverlaySingleDocTermsLeafReader.java        | 113 ++++++++++
 .../lucene/search/uhighlight/Passage.java       |  11 +-
 .../uhighlight/PostingsOffsetStrategy.java      |  24 +--
 .../PostingsWithTermVectorsOffsetStrategy.java  |  26 +--
 .../TermVectorFilteredLeafReader.java           |  11 +-
 .../uhighlight/TermVectorOffsetStrategy.java    |  23 +-
 .../uhighlight/TokenStreamOffsetStrategy.java   |  20 +-
 .../lucene/search/uhighlight/UHComponents.java  |  80 +++++++
 .../search/uhighlight/UnifiedHighlighter.java   |  73 +++++--
 .../uhighlight/TestUnifiedHighlighter.java      | 154 ++++++++++---
 .../uhighlight/TestUnifiedHighlighterMTQ.java   |   2 +-
 .../TestUnifiedHighlighterRanking.java          |  20 ++
 .../TestUnifiedHighlighterStrictPhrases.java    | 146 +++++++++----
 .../TestUnifiedHighlighterTermVec.java          |  34 ++-
 .../TestUnifiedHighlighterExtensibility.java    |  36 +++-
 22 files changed, 947 insertions(+), 250 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b19ae942/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 413dda5..e95abda 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -186,6 +186,13 @@ API Changes:
 * LUCENE-8437: CheckIndex.Status.cantOpenSegments and missingSegmentVersion
   have been removed as they were not computed correctly. (Adrien Grand)
 
+* LUCENE-8286: The UnifiedHighlighter has a new HighlightFlag.WEIGHT_MATCHES flag that
+  will tell this highlighter to use the new MatchesIterator API as the underlying
+  approach to navigate matching hits for a query.  This mode will highlight more
+  accurately than any other highlighter, and can mark up phrases as one span instead of
+  word-by-word.  The UH's public internal APIs changed a bit in the process.
+  (David Smiley)
+
 Bug Fixes:
 
 * LUCENE-8445: Tighten condition when two planes are identical to prevent constructing

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b19ae942/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/AnalysisOffsetStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/AnalysisOffsetStrategy.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/AnalysisOffsetStrategy.java
index 162d270..63785e3 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/AnalysisOffsetStrategy.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/AnalysisOffsetStrategy.java
@@ -23,8 +23,6 @@ import org.apache.lucene.analysis.TokenFilter;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.automaton.CharacterRunAutomaton;
 
 /**
  * Provides a base class for analysis based offset strategies to extend from.
@@ -37,12 +35,12 @@ public abstract class AnalysisOffsetStrategy extends FieldOffsetStrategy {
 
   protected final Analyzer analyzer;
 
-  public AnalysisOffsetStrategy(String field, BytesRef[] queryTerms, PhraseHelper phraseHelper, CharacterRunAutomaton[] automata, Analyzer analyzer) {
-    super(field, queryTerms, phraseHelper, automata);
+  public AnalysisOffsetStrategy(UHComponents components, Analyzer analyzer) {
+    super(components);
     this.analyzer = analyzer;
-    if (analyzer.getOffsetGap(field) != 1) { // note: 1 is the default. It is RARELY changed.
+    if (analyzer.getOffsetGap(getField()) != 1) { // note: 1 is the default. It is RARELY changed.
       throw new IllegalArgumentException(
-          "offset gap of the provided analyzer should be 1 (field " + field + ")");
+          "offset gap of the provided analyzer should be 1 (field " + getField() + ")");
     }
   }
 
@@ -55,12 +53,12 @@ public abstract class AnalysisOffsetStrategy extends FieldOffsetStrategy {
     // If there is no splitChar in content then we needn't wrap:
     int splitCharIdx = content.indexOf(UnifiedHighlighter.MULTIVAL_SEP_CHAR);
     if (splitCharIdx == -1) {
-      return analyzer.tokenStream(field, content);
+      return analyzer.tokenStream(getField(), content);
     }
 
-    TokenStream subTokenStream = analyzer.tokenStream(field, content.substring(0, splitCharIdx));
+    TokenStream subTokenStream = analyzer.tokenStream(getField(), content.substring(0, splitCharIdx));
 
-    return new MultiValueTokenStream(subTokenStream, field, analyzer, content, UnifiedHighlighter.MULTIVAL_SEP_CHAR, splitCharIdx);
+    return new MultiValueTokenStream(subTokenStream, getField(), analyzer, content, UnifiedHighlighter.MULTIVAL_SEP_CHAR, splitCharIdx);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b19ae942/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/FieldHighlighter.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/FieldHighlighter.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/FieldHighlighter.java
index da6e525..f2a34dc 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/FieldHighlighter.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/FieldHighlighter.java
@@ -24,7 +24,7 @@ import java.util.Comparator;
 import java.util.List;
 import java.util.PriorityQueue;
 
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.util.BytesRef;
 
 /**
@@ -65,8 +65,7 @@ public class FieldHighlighter {
   /**
    * The primary method -- highlight this doc, assuming a specific field and given this content.
    */
-  public Object highlightFieldForDoc(IndexReader reader, int docId, String content) throws IOException {
-    // TODO accept LeafReader instead?
+  public Object highlightFieldForDoc(LeafReader reader, int docId, String content) throws IOException {
     // note: it'd be nice to accept a CharSequence for content, but we need a CharacterIterator impl for it.
     if (content.length() == 0) {
       return null; // nothing to do

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b19ae942/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/FieldOffsetStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/FieldOffsetStrategy.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/FieldOffsetStrategy.java
index cf564f4..c63896a 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/FieldOffsetStrategy.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/FieldOffsetStrategy.java
@@ -20,11 +20,15 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.FilterLeafReader;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Matches;
+import org.apache.lucene.search.MatchesIterator;
+import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRefBuilder;
 import org.apache.lucene.util.automaton.CharacterRunAutomaton;
@@ -37,20 +41,14 @@ import org.apache.lucene.util.automaton.CharacterRunAutomaton;
  */
 public abstract class FieldOffsetStrategy {
 
-  protected final String field;
-  protected final PhraseHelper phraseHelper; // Query: position-sensitive information
-  protected final BytesRef[] terms; // Query: all terms we extracted (some may be position sensitive)
-  protected final CharacterRunAutomaton[] automata; // Query: wildcards (i.e. multi-term query), not position sensitive
+  protected final UHComponents components;
 
-  public FieldOffsetStrategy(String field, BytesRef[] queryTerms, PhraseHelper phraseHelper, CharacterRunAutomaton[] automata) {
-    this.field = field;
-    this.terms = queryTerms;
-    this.phraseHelper = phraseHelper;
-    this.automata = automata;
+  public FieldOffsetStrategy(UHComponents components) {
+    this.components = components;
   }
 
   public String getField() {
-    return field;
+    return components.getField();
   }
 
   public abstract UnifiedHighlighter.OffsetSource getOffsetSource();
@@ -60,39 +58,97 @@ public abstract class FieldOffsetStrategy {
    *
    * Callers are expected to close the returned OffsetsEnum when it has been finished with
    */
-  public abstract OffsetsEnum getOffsetsEnum(IndexReader reader, int docId, String content) throws IOException;
+  public abstract OffsetsEnum getOffsetsEnum(LeafReader reader, int docId, String content) throws IOException;
 
   protected OffsetsEnum createOffsetsEnumFromReader(LeafReader leafReader, int doc) throws IOException {
-    final Terms termsIndex = leafReader.terms(field);
+    final Terms termsIndex = leafReader.terms(getField());
     if (termsIndex == null) {
       return OffsetsEnum.EMPTY;
     }
 
-    final List<OffsetsEnum> offsetsEnums = new ArrayList<>(terms.length + automata.length);
+    final List<OffsetsEnum> offsetsEnums = new ArrayList<>();
 
-    // Handle position insensitive terms (a subset of this.terms field):
-    final BytesRef[] insensitiveTerms;
-    if (phraseHelper.hasPositionSensitivity()) {
-      insensitiveTerms = phraseHelper.getAllPositionInsensitiveTerms();
-      assert insensitiveTerms.length <= terms.length : "insensitive terms should be smaller set of all terms";
-    } else {
-      insensitiveTerms = terms;
-    }
-    if (insensitiveTerms.length > 0) {
-      createOffsetsEnumsForTerms(insensitiveTerms, termsIndex, doc, offsetsEnums);
+    // Handle Weight.matches approach
+    if (components.getHighlightFlags().contains(UnifiedHighlighter.HighlightFlag.WEIGHT_MATCHES)) {
+
+      createOffsetsEnumsWeightMatcher(leafReader, doc, offsetsEnums);
+
+    } else { // classic approach
+
+      // Handle position insensitive terms (a subset of this.terms field):
+      final BytesRef[] insensitiveTerms;
+      final PhraseHelper phraseHelper = components.getPhraseHelper();
+      final BytesRef[] terms = components.getTerms();
+      if (phraseHelper.hasPositionSensitivity()) {
+        insensitiveTerms = phraseHelper.getAllPositionInsensitiveTerms();
+        assert insensitiveTerms.length <= terms.length : "insensitive terms should be smaller set of all terms";
+      } else {
+        insensitiveTerms = terms;
+      }
+      if (insensitiveTerms.length > 0) {
+        createOffsetsEnumsForTerms(insensitiveTerms, termsIndex, doc, offsetsEnums);
+      }
+
+      // Handle spans
+      if (phraseHelper.hasPositionSensitivity()) {
+        phraseHelper.createOffsetsEnumsForSpans(leafReader, doc, offsetsEnums);
+      }
+
+      // Handle automata
+      if (components.getAutomata().length > 0) {
+        createOffsetsEnumsForAutomata(termsIndex, doc, offsetsEnums);
+      }
     }
 
-    // Handle spans
-    if (phraseHelper.hasPositionSensitivity()) {
-      phraseHelper.createOffsetsEnumsForSpans(leafReader, doc, offsetsEnums);
+    switch (offsetsEnums.size()) {
+      case 0: return OffsetsEnum.EMPTY;
+      case 1: return offsetsEnums.get(0);
+      default: return new OffsetsEnum.MultiOffsetsEnum(offsetsEnums);
     }
+  }
+
+  protected void createOffsetsEnumsWeightMatcher(LeafReader _leafReader, int docId, List<OffsetsEnum> results) throws IOException {
+    // remap fieldMatcher/requireFieldMatch fields to the field we are highlighting
+    LeafReader leafReader = new FilterLeafReader(_leafReader) {
+      @Override
+      public Terms terms(String field) throws IOException {
+        if (components.getFieldMatcher().test(field)) {
+          return super.terms(components.getField());
+        } else {
+          return super.terms(field);
+        }
+      }
+
+      //  So many subclasses do this!
+      //these ought to be a default or added via some intermediary like "FilterTransientLeafReader" (exception on close).
+      @Override
+      public CacheHelper getCoreCacheHelper() {
+        return null;
+      }
 
-    // Handle automata
-    if (automata.length > 0) {
-      createOffsetsEnumsForAutomata(termsIndex, doc, offsetsEnums);
+      @Override
+      public CacheHelper getReaderCacheHelper() {
+        return null;
+      }
+    };
+    IndexSearcher indexSearcher = new IndexSearcher(leafReader);
+    indexSearcher.setQueryCache(null);
+    Matches matches = indexSearcher.rewrite(components.getQuery())
+        .createWeight(indexSearcher, ScoreMode.COMPLETE_NO_SCORES, 1.0f)
+        .matches(leafReader.getContext(), docId);
+    if (matches == null) {
+      return; // doc doesn't match
+    }
+    for (String field : matches) {
+      if (components.getFieldMatcher().test(field)) {
+        MatchesIterator iterator = matches.getMatches(field);
+        if (iterator == null) {
+          continue;
+        }
+        results.add(new OffsetsEnum.OfMatchesIteratorWithSubs(iterator));
+      }
     }
 
-    return new OffsetsEnum.MultiOffsetsEnum(offsetsEnums);
   }
 
   protected void createOffsetsEnumsForTerms(BytesRef[] sourceTerms, Terms termsIndex, int doc, List<OffsetsEnum> results) throws IOException {
@@ -102,7 +158,7 @@ public abstract class FieldOffsetStrategy {
         PostingsEnum postingsEnum = termsEnum.postings(null, PostingsEnum.OFFSETS);
         if (postingsEnum == null) {
           // no offsets or positions available
-          throw new IllegalArgumentException("field '" + field + "' was indexed without offsets, cannot highlight");
+          throw new IllegalArgumentException("field '" + getField() + "' was indexed without offsets, cannot highlight");
         }
         if (doc == postingsEnum.advance(doc)) { // now it's positioned, although may be exhausted
           results.add(new OffsetsEnum.OfPostings(term, postingsEnum));
@@ -112,6 +168,7 @@ public abstract class FieldOffsetStrategy {
   }
 
   protected void createOffsetsEnumsForAutomata(Terms termsIndex, int doc, List<OffsetsEnum> results) throws IOException {
+    final CharacterRunAutomaton[] automata = components.getAutomata();
     List<List<PostingsEnum>> automataPostings = new ArrayList<>(automata.length);
     for (int i = 0; i < automata.length; i++) {
       automataPostings.add(new ArrayList<>());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b19ae942/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 6364f3f..30dbdd4 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
@@ -23,18 +23,15 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.function.Function;
-import java.util.function.Predicate;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.FilteringTokenFilter;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
-import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.memory.MemoryIndex;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.spans.SpanQuery;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.automaton.Automata;
 import org.apache.lucene.util.automaton.CharacterRunAutomaton;
 
@@ -47,36 +44,32 @@ import org.apache.lucene.util.automaton.CharacterRunAutomaton;
 public class MemoryIndexOffsetStrategy extends AnalysisOffsetStrategy {
 
   private final MemoryIndex memoryIndex;
-  private final LeafReader leafReader;
+  private final LeafReader memIndexLeafReader;
   private final CharacterRunAutomaton preMemIndexFilterAutomaton;
 
-  public MemoryIndexOffsetStrategy(String field, Predicate<String> fieldMatcher, BytesRef[] extractedTerms, PhraseHelper phraseHelper,
-                                   CharacterRunAutomaton[] automata, Analyzer analyzer,
+  public MemoryIndexOffsetStrategy(UHComponents components, Analyzer analyzer,
                                    Function<Query, Collection<Query>> multiTermQueryRewrite) {
-    super(field, extractedTerms, phraseHelper, automata, analyzer);
-    boolean storePayloads = phraseHelper.hasPositionSensitivity(); // might be needed
+    super(components, analyzer);
+    boolean storePayloads = components.getPhraseHelper().hasPositionSensitivity(); // might be needed
     memoryIndex = new MemoryIndex(true, storePayloads);//true==store offsets
-    leafReader = (LeafReader) memoryIndex.createSearcher().getIndexReader(); // appears to be re-usable
+    memIndexLeafReader = (LeafReader) memoryIndex.createSearcher().getIndexReader(); // appears to be re-usable
     // preFilter for MemoryIndex
-    preMemIndexFilterAutomaton = buildCombinedAutomaton(fieldMatcher, terms, this.automata, phraseHelper, multiTermQueryRewrite);
+    preMemIndexFilterAutomaton = buildCombinedAutomaton(components, multiTermQueryRewrite);
   }
 
   /**
    * Build one {@link CharacterRunAutomaton} matching any term the query might match.
    */
-  private static CharacterRunAutomaton buildCombinedAutomaton(Predicate<String> fieldMatcher,
-                                                              BytesRef[] terms,
-                                                              CharacterRunAutomaton[] automata,
-                                                              PhraseHelper strictPhrases,
+  private static CharacterRunAutomaton buildCombinedAutomaton(UHComponents components,
                                                               Function<Query, Collection<Query>> multiTermQueryRewrite) {
     List<CharacterRunAutomaton> allAutomata = new ArrayList<>();
-    if (terms.length > 0) {
-      allAutomata.add(new CharacterRunAutomaton(Automata.makeStringUnion(Arrays.asList(terms))));
+    if (components.getTerms().length > 0) {
+      allAutomata.add(new CharacterRunAutomaton(Automata.makeStringUnion(Arrays.asList(components.getTerms()))));
     }
-    Collections.addAll(allAutomata, automata);
-    for (SpanQuery spanQuery : strictPhrases.getSpanQueries()) {
+    Collections.addAll(allAutomata, components.getAutomata());
+    for (SpanQuery spanQuery : components.getPhraseHelper().getSpanQueries()) {
       Collections.addAll(allAutomata,
-          MultiTermHighlighting.extractAutomata(spanQuery, fieldMatcher, true, multiTermQueryRewrite));//true==lookInSpan
+          MultiTermHighlighting.extractAutomata(spanQuery, components.getFieldMatcher(), true, multiTermQueryRewrite));//true==lookInSpan
     }
 
     if (allAutomata.size() == 1) {
@@ -100,20 +93,28 @@ public class MemoryIndexOffsetStrategy extends AnalysisOffsetStrategy {
   }
 
   @Override
-  public OffsetsEnum getOffsetsEnum(IndexReader reader, int docId, String content) throws IOException {
+  public OffsetsEnum getOffsetsEnum(LeafReader reader, int docId, String content) throws IOException {
     // note: don't need LimitTokenOffsetFilter since content is already truncated to maxLength
     TokenStream tokenStream = tokenStream(content);
 
     // Filter the tokenStream to applicable terms
     tokenStream = newKeepWordFilter(tokenStream, preMemIndexFilterAutomaton);
     memoryIndex.reset();
-    memoryIndex.addField(field, tokenStream);//note: calls tokenStream.reset() & close()
-    docId = 0;
-
-    return createOffsetsEnumFromReader(leafReader, docId);
+    memoryIndex.addField(getField(), tokenStream);//note: calls tokenStream.reset() & close()
+
+    if (reader == null) {
+      return createOffsetsEnumFromReader(memIndexLeafReader, 0);
+    } else {
+      return createOffsetsEnumFromReader(
+          new OverlaySingleDocTermsLeafReader(
+              reader,
+              memIndexLeafReader,
+              getField(),
+              docId),
+          docId);
+    }
   }
 
-
   private static FilteringTokenFilter newKeepWordFilter(final TokenStream tokenStream,
                                                         final CharacterRunAutomaton charRunAutomaton) {
     // it'd be nice to use KeepWordFilter but it demands a CharArraySet. TODO File JIRA? Need a new interface?

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b19ae942/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/NoOpOffsetStrategy.java
----------------------------------------------------------------------
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 44a23f7..80528ce 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
@@ -17,8 +17,10 @@
 package org.apache.lucene.search.uhighlight;
 
 import java.io.IOException;
+import java.util.Collections;
 
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.search.MatchNoDocsQuery;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.automaton.CharacterRunAutomaton;
 
@@ -32,7 +34,7 @@ public class NoOpOffsetStrategy extends FieldOffsetStrategy {
   public static final NoOpOffsetStrategy INSTANCE = new NoOpOffsetStrategy();
 
   private NoOpOffsetStrategy() {
-    super("_ignored_", new BytesRef[0], PhraseHelper.NONE, new CharacterRunAutomaton[0]);
+    super(new UHComponents("_ignored_", (s) -> false, new MatchNoDocsQuery(), new BytesRef[0], PhraseHelper.NONE, new CharacterRunAutomaton[0], Collections.emptySet()));
   }
 
   @Override
@@ -41,7 +43,7 @@ public class NoOpOffsetStrategy extends FieldOffsetStrategy {
   }
 
   @Override
-  public OffsetsEnum getOffsetsEnum(IndexReader reader, int docId, String content) throws IOException {
+  public OffsetsEnum getOffsetsEnum(LeafReader reader, int docId, String content) throws IOException {
     return OffsetsEnum.EMPTY;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b19ae942/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/OffsetsEnum.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/OffsetsEnum.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/OffsetsEnum.java
index bdabcc1..0e6a221 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/OffsetsEnum.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/OffsetsEnum.java
@@ -19,12 +19,20 @@ package org.apache.lucene.search.uhighlight;
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Objects;
 import java.util.PriorityQueue;
+import java.util.TreeSet;
+import java.util.function.Supplier;
 
 import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.MatchesIterator;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IOUtils;
 
 /**
@@ -77,8 +85,10 @@ public abstract class OffsetsEnum implements Comparable<OffsetsEnum>, Closeable
   public abstract int freq() throws IOException;
 
   /**
-   * The term at this position; usually always the same.
+   * The term at this position.
    * This BytesRef is safe to continue to refer to, even after we move to the next position.
+   *
+   * @see Passage#getMatchTerms()
    */
   public abstract BytesRef getTerm() throws IOException;
 
@@ -93,8 +103,14 @@ public abstract class OffsetsEnum implements Comparable<OffsetsEnum>, Closeable
   @Override
   public String toString() {
     final String name = getClass().getSimpleName();
+    String offset = "";
     try {
-      return name + "(term:" + getTerm().utf8ToString() +")";
+      offset = ",[" + startOffset() + "-" + endOffset() + "]";
+    } catch (Exception e) {
+      //ignore; for debugging only
+    }
+    try {
+      return name + "(term:" + getTerm().utf8ToString() + offset + ")";
     } catch (Exception e) {
       return name;
     }
@@ -157,6 +173,201 @@ public abstract class OffsetsEnum implements Comparable<OffsetsEnum>, Closeable
     }
   }
 
+  /** Based on a {@link MatchesIterator} with submatches. */
+  public static class OfMatchesIteratorWithSubs extends OffsetsEnum {
+    //Either CachedOE impls (which are the submatches) or OfMatchesIterator impls
+    private final PriorityQueue<OffsetsEnum> pendingQueue = new PriorityQueue<>();
+    private final HashMap<Query,BytesRef> queryToTermMap = new HashMap<>();
+
+    public OfMatchesIteratorWithSubs(MatchesIterator matchesIterator) {
+      pendingQueue.add(new OfMatchesIterator(matchesIterator, () -> queryToTerm(matchesIterator.getQuery())));
+    }
+
+    @Override
+    public boolean nextPosition() throws IOException {
+      OffsetsEnum formerHeadOE = pendingQueue.poll(); // removes the head
+      if (formerHeadOE instanceof CachedOE) {
+        // we're done with the former head.  CachedOE's are one use only.
+        // Look at the new head...
+        OffsetsEnum newHeadOE = pendingQueue.peek();
+        if (newHeadOE instanceof OfMatchesIterator) {
+          // We found the matchesIterator.  Requires processing.
+          nextWhenMatchesIterator((OfMatchesIterator) newHeadOE);  // May or may not remove or re-queue itself
+        } // else new head is a CacheOE or no more.  Nothing to do with it.
+
+      } else { // formerHeadOE is OfMatchesIterator; advance it
+        OfMatchesIterator miOE = (OfMatchesIterator) formerHeadOE;
+        if (miOE.nextPosition()) {
+          nextWhenMatchesIterator(miOE); // requires processing.  May or may not re-enqueue itself
+        }
+      }
+      return pendingQueue.isEmpty() == false;
+    }
+
+    private void nextWhenMatchesIterator(OfMatchesIterator miOE) throws IOException {
+      boolean isHead = miOE == pendingQueue.peek();
+      MatchesIterator subMatches = miOE.matchesIterator.getSubMatches();
+      if (subMatches != null) {
+        // remove this miOE from the queue, add it's submatches, next() it, then re-enqueue it
+        if (isHead) {
+          pendingQueue.poll(); // remove
+        }
+
+        enqueueCachedMatches(subMatches);
+
+        if (miOE.nextPosition()) {
+          pendingQueue.add(miOE);
+          assert pendingQueue.peek() != miOE; // miOE should follow cached entries
+        }
+
+      } else { // else has no subMatches.  It will stay enqueued.
+        if (!isHead) {
+          pendingQueue.add(miOE);
+        } // else it's *already* in pendingQueue
+      }
+    }
+
+    private boolean enqueueCachedMatches(MatchesIterator thisMI) throws IOException {
+      if (thisMI == null) {
+        return false;
+      } else {
+        while (thisMI.next()) {
+          if (false == enqueueCachedMatches(thisMI.getSubMatches())) { // recursion
+            // if no sub-matches then add ourselves
+            pendingQueue.add(new CachedOE(queryToTerm(thisMI.getQuery()), thisMI.startOffset(), thisMI.endOffset()));
+          }
+        }
+        return true;
+      }
+    }
+
+    /** Maps a Query from {@link MatchesIterator#getQuery()} to {@link OffsetsEnum#getTerm()}.
+     * See {@link Passage#getMatchTerms()}. */
+    private BytesRef queryToTerm(Query query) {
+      // compute an approximate BytesRef term of a Query.  We cache this since we're likely to see the same query again.
+      // Our approach is to call extractTerms and visit each term in order, concatenating them with an adjoining space.
+      //  If we don't have any (perhaps due to an MTQ like a wildcard) then we fall back on the toString() of the query.
+      return queryToTermMap.computeIfAbsent(query, (Query q) -> {
+        try {
+          BytesRefBuilder bytesRefBuilder = new BytesRefBuilder();
+          UnifiedHighlighter.EMPTY_INDEXSEARCHER
+              .createWeight(UnifiedHighlighter.EMPTY_INDEXSEARCHER.rewrite(q), ScoreMode.COMPLETE_NO_SCORES, 1f)
+              .extractTerms(new TreeSet<Term>() {
+            @Override
+            public boolean add(Term term) {
+              if (bytesRefBuilder.length() > 0) {
+                bytesRefBuilder.append((byte) ' ');
+              }
+              bytesRefBuilder.append(term.bytes());
+              return true;
+            }
+          });
+          if (bytesRefBuilder.length() > 0) {
+            return bytesRefBuilder.get();
+          }
+        } catch (IOException e) {//ignore
+          // go to fallback...
+        }
+
+        // fallback:  (likely a MultiTermQuery)
+        return new BytesRef(q.toString());
+      });
+    }
+
+    @Override
+    public int freq() throws IOException {
+      return pendingQueue.peek().freq();
+    }
+
+    @Override
+    public BytesRef getTerm() throws IOException {
+      return pendingQueue.peek().getTerm();
+    }
+
+    @Override
+    public int startOffset() throws IOException {
+      return pendingQueue.peek().startOffset();
+    }
+
+    @Override
+    public int endOffset() throws IOException {
+      return pendingQueue.peek().endOffset();
+    }
+
+    private static class CachedOE extends OffsetsEnum {
+      final BytesRef term;
+      final int startOffset;
+      final int endOffset;
+
+      private CachedOE(BytesRef term, int startOffset, int endOffset) {
+        this.term = term;
+        this.startOffset = startOffset;
+        this.endOffset = endOffset;
+      }
+
+      @Override
+      public boolean nextPosition() throws IOException {
+        return false;
+      }
+
+      @Override
+      public int freq() throws IOException {
+        return 1; // documented short-coming of MatchesIterator based UnifiedHighlighter
+      }
+
+      @Override
+      public BytesRef getTerm() throws IOException {
+        return term;
+      }
+
+      @Override
+      public int startOffset() throws IOException {
+        return startOffset;
+      }
+
+      @Override
+      public int endOffset() throws IOException {
+        return endOffset;
+      }
+    }
+  }
+
+  /** Based on a {@link MatchesIterator}; does not look at submatches. */
+  public static class OfMatchesIterator extends OffsetsEnum {
+    private final MatchesIterator matchesIterator;
+    private final Supplier<BytesRef> termSupplier;
+
+    public OfMatchesIterator(MatchesIterator matchesIterator, Supplier<BytesRef> termSupplier) {
+      this.matchesIterator = matchesIterator;
+      this.termSupplier = termSupplier;
+    }
+
+    @Override
+    public boolean nextPosition() throws IOException {
+      return matchesIterator.next();
+    }
+
+    @Override
+    public int freq() throws IOException {
+      return 1; // documented short-coming of MatchesIterator based UnifiedHighlighter
+    }
+
+    @Override
+    public BytesRef getTerm() throws IOException {
+      return termSupplier.get();
+    }
+
+    @Override
+    public int startOffset() throws IOException {
+      return matchesIterator.startOffset();
+    }
+
+    @Override
+    public int endOffset() throws IOException {
+      return matchesIterator.endOffset();
+    }
+  }
+
   /**
    * Empty enumeration
    */
@@ -191,6 +402,7 @@ public abstract class OffsetsEnum implements Comparable<OffsetsEnum>, Closeable
   /**
    * A view over several OffsetsEnum instances, merging them in-place
    */
+  //If OffsetsEnum and MatchesIterator ever truly merge then this could go away in lieu of DisjunctionMatchesIterator
   public static class MultiOffsetsEnum extends OffsetsEnum {
 
     private final PriorityQueue<OffsetsEnum> queue;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b19ae942/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/OverlaySingleDocTermsLeafReader.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/OverlaySingleDocTermsLeafReader.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/OverlaySingleDocTermsLeafReader.java
new file mode 100644
index 0000000..53f29e3
--- /dev/null
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/OverlaySingleDocTermsLeafReader.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search.uhighlight;
+
+import java.io.IOException;
+
+import org.apache.lucene.index.FilterLeafReader;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
+
+/**
+ * Overlays a 2nd LeafReader for the terms of one field, otherwise the primary reader is
+ * consulted.  The 2nd reader is assumed to have one document of 0 and we remap it to a target doc ID.
+ *
+ * @lucene.internal
+ */
+public class OverlaySingleDocTermsLeafReader extends FilterLeafReader {
+
+  private final LeafReader in2;
+  private final String in2Field;
+  private final int in2TargetDocId;
+
+  public OverlaySingleDocTermsLeafReader(LeafReader in, LeafReader in2, String in2Field, int in2TargetDocId) {
+    super(in);
+    this.in2 = in2;
+    this.in2Field = in2Field;
+    this.in2TargetDocId = in2TargetDocId;
+    assert in2.maxDoc() == 1;
+  }
+
+  @Override
+  public Terms terms(String field) throws IOException {
+    if (!in2Field.equals(field)) {
+      return in.terms(field);
+    }
+
+    // Shifts leafReader in2 with only doc ID 0 to a target doc ID
+    final Terms terms = in2.terms(field);
+    if (terms == null) {
+      return null;
+    }
+    if (in2TargetDocId == 0) { // no doc ID remapping to do
+      return terms;
+    }
+    return new FilterTerms(terms) {
+      @Override
+      public TermsEnum iterator() throws IOException {
+        return filterTermsEnum(super.iterator());
+      }
+
+      @Override
+      public TermsEnum intersect(CompiledAutomaton compiled, BytesRef startTerm) throws IOException {
+        return filterTermsEnum(super.intersect(compiled, startTerm));
+      }
+
+      private TermsEnum filterTermsEnum(TermsEnum termsEnum) throws IOException {
+        return new FilterTermsEnum(termsEnum) {
+          @Override
+          public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
+            //TODO 'reuse' will always fail to reuse unless we unwrap it
+            return new FilterPostingsEnum(super.postings(reuse, flags)) {
+              @Override
+              public int nextDoc() throws IOException {
+                final int doc = super.nextDoc();
+                return doc == 0 ? in2TargetDocId : doc;
+              }
+
+              @Override
+              public int advance(int target) throws IOException {
+                return slowAdvance(target);
+              }
+
+              @Override
+              public int docID() {
+                final int doc = super.docID();
+                return doc == 0 ? in2TargetDocId : doc;
+              }
+            };
+          }
+        };
+      }
+    };
+  }
+
+  @Override
+  public CacheHelper getCoreCacheHelper() {
+    return null;
+  }
+
+  @Override
+  public CacheHelper getReaderCacheHelper() {
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b19ae942/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/Passage.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/Passage.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/Passage.java
index 570fcdf..1c760cb 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/Passage.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/Passage.java
@@ -157,7 +157,16 @@ public class Passage {
   }
 
   /**
-   * BytesRef (term text) of the matches, corresponding with {@link #getMatchStarts()}.
+   * BytesRef (term text) of the matches, corresponding with {@link #getMatchStarts()}.  The primary purpose of this
+   * method is to expose the number of unique terms per passage for use in passage scoring.
+   * The actual term byte content is not well defined by this highlighter, and thus use of it is more subject to
+   * change.
+   * <p>
+   * The term might be simply the analyzed term at this position.
+   * Depending on the highlighter's configuration, the match term may be a phrase (instead of a word), and in such
+   * a case might be a series of space-separated analyzed terms.
+   * If the match is from a {@link org.apache.lucene.search.MultiTermQuery} then the match term may be the toString() of
+   * that query.
    * <p>
    * Only {@link #getNumMatches()} are valid.
    */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b19ae942/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/PostingsOffsetStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/PostingsOffsetStrategy.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/PostingsOffsetStrategy.java
index b7df77a..53c1c95 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/PostingsOffsetStrategy.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/PostingsOffsetStrategy.java
@@ -17,15 +17,9 @@
 package org.apache.lucene.search.uhighlight;
 
 import java.io.IOException;
-import java.util.List;
 
 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.ReaderUtil;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.automaton.CharacterRunAutomaton;
 
 /**
  * Uses offsets in postings -- {@link IndexOptions#DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS}.  This
@@ -35,23 +29,13 @@ import org.apache.lucene.util.automaton.CharacterRunAutomaton;
  */
 public class PostingsOffsetStrategy extends FieldOffsetStrategy {
 
-  public PostingsOffsetStrategy(String field, BytesRef[] queryTerms, PhraseHelper phraseHelper, CharacterRunAutomaton[] automata) {
-    super(field, queryTerms, phraseHelper, automata);
+  public PostingsOffsetStrategy(UHComponents components) {
+    super(components);
   }
 
   @Override
-  public OffsetsEnum getOffsetsEnum(IndexReader reader, int docId, String content) throws IOException {
-    final LeafReader leafReader;
-    if (reader instanceof LeafReader) {
-      leafReader = (LeafReader) reader;
-    } else {
-      List<LeafReaderContext> leaves = reader.leaves();
-      LeafReaderContext leafReaderContext = leaves.get(ReaderUtil.subIndex(docId, leaves));
-      leafReader = leafReaderContext.reader();
-      docId -= leafReaderContext.docBase; // adjust 'doc' to be within this leaf reader
-    }
-
-    return createOffsetsEnumFromReader(leafReader, docId);
+  public OffsetsEnum getOffsetsEnum(LeafReader reader, int docId, String content) throws IOException {
+    return createOffsetsEnumFromReader(reader, docId);
   }
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b19ae942/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/PostingsWithTermVectorsOffsetStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/PostingsWithTermVectorsOffsetStrategy.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/PostingsWithTermVectorsOffsetStrategy.java
index 9097e4d..b0249fa 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/PostingsWithTermVectorsOffsetStrategy.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/PostingsWithTermVectorsOffsetStrategy.java
@@ -17,15 +17,9 @@
 package org.apache.lucene.search.uhighlight;
 
 import java.io.IOException;
-import java.util.List;
 
-import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.ReaderUtil;
 import org.apache.lucene.index.Terms;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.automaton.CharacterRunAutomaton;
 
 /**
  * Like {@link PostingsOffsetStrategy} but also uses term vectors (only terms needed) for multi-term queries.
@@ -34,27 +28,17 @@ import org.apache.lucene.util.automaton.CharacterRunAutomaton;
  */
 public class PostingsWithTermVectorsOffsetStrategy extends FieldOffsetStrategy {
 
-  public PostingsWithTermVectorsOffsetStrategy(String field, BytesRef[] queryTerms, PhraseHelper phraseHelper, CharacterRunAutomaton[] automata) {
-    super(field, queryTerms, phraseHelper, automata);
+  public PostingsWithTermVectorsOffsetStrategy(UHComponents components) {
+    super(components);
   }
 
   @Override
-  public OffsetsEnum getOffsetsEnum(IndexReader reader, int docId, String content) throws IOException {
-    LeafReader leafReader;
-    if (reader instanceof LeafReader) {
-      leafReader = (LeafReader) reader;
-    } else {
-      List<LeafReaderContext> leaves = reader.leaves();
-      LeafReaderContext LeafReaderContext = leaves.get(ReaderUtil.subIndex(docId, leaves));
-      leafReader = LeafReaderContext.reader();
-      docId -= LeafReaderContext.docBase; // adjust 'doc' to be within this atomic reader
-    }
-
-    Terms docTerms = leafReader.getTermVector(docId, field);
+  public OffsetsEnum getOffsetsEnum(LeafReader leafReader, int docId, String content) throws IOException {
+    Terms docTerms = leafReader.getTermVector(docId, getField());
     if (docTerms == null) {
       return OffsetsEnum.EMPTY;
     }
-    leafReader = new TermVectorFilteredLeafReader(leafReader, docTerms);
+    leafReader = new TermVectorFilteredLeafReader(leafReader, docTerms, getField());
 
     return createOffsetsEnumFromReader(leafReader, docId);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b19ae942/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/TermVectorFilteredLeafReader.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/TermVectorFilteredLeafReader.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/TermVectorFilteredLeafReader.java
index 4165af4..4b93e16 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/TermVectorFilteredLeafReader.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/TermVectorFilteredLeafReader.java
@@ -37,21 +37,26 @@ final class TermVectorFilteredLeafReader extends FilterLeafReader {
   // NOTE: super ("in") is baseLeafReader
 
   private final Terms filterTerms;
+  private final String fieldFilter;
 
   /**
    * <p>Construct a FilterLeafReader based on the specified base reader.
    * <p>Note that base reader is closed if this FilterLeafReader is closed.</p>
-   *
    * @param baseLeafReader full/original reader.
    * @param filterTerms set of terms to filter by -- probably from a TermVector or MemoryIndex.
+   * @param fieldFilter the field to do this on
    */
-  TermVectorFilteredLeafReader(LeafReader baseLeafReader, Terms filterTerms) {
+  TermVectorFilteredLeafReader(LeafReader baseLeafReader, Terms filterTerms, String fieldFilter) {
     super(baseLeafReader);
     this.filterTerms = filterTerms;
+    this.fieldFilter = fieldFilter;
   }
 
   @Override
   public Terms terms(String field) throws IOException {
+    if (!field.equals(fieldFilter)) {
+      return super.terms(field); // proceed like normal for fields we're not interested in
+    }
     Terms terms = in.terms(field);
     return terms==null ? null : new TermsFilteredTerms(terms, filterTerms);
   }
@@ -106,7 +111,7 @@ final class TermVectorFilteredLeafReader extends FilterLeafReader {
       boolean termInBothTermsEnum = baseTermsEnum.seekExact(currentTerm);
 
       if (!termInBothTermsEnum) {
-        throw new IllegalStateException("Term vector term " + currentTerm + " does not appear in full index.");
+        throw new IllegalStateException("Term vector term '" + currentTerm.utf8ToString() + "' does not appear in full index.");
       }
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b19ae942/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/TermVectorOffsetStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/TermVectorOffsetStrategy.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/TermVectorOffsetStrategy.java
index cd19bb9..94528e2 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/TermVectorOffsetStrategy.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/TermVectorOffsetStrategy.java
@@ -18,12 +18,9 @@ package org.apache.lucene.search.uhighlight;
 
 import java.io.IOException;
 
-import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.search.highlight.TermVectorLeafReader;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.automaton.CharacterRunAutomaton;
 
 /**
  * Uses term vectors that contain offsets.
@@ -32,8 +29,8 @@ import org.apache.lucene.util.automaton.CharacterRunAutomaton;
  */
 public class TermVectorOffsetStrategy extends FieldOffsetStrategy {
 
-  public TermVectorOffsetStrategy(String field, BytesRef[] queryTerms, PhraseHelper phraseHelper, CharacterRunAutomaton[] automata) {
-    super(field, queryTerms, phraseHelper, automata);
+  public TermVectorOffsetStrategy(UHComponents components) {
+    super(components);
   }
 
   @Override
@@ -42,16 +39,20 @@ public class TermVectorOffsetStrategy extends FieldOffsetStrategy {
   }
 
   @Override
-  public OffsetsEnum getOffsetsEnum(IndexReader reader, int docId, String content) throws IOException {
-    Terms tvTerms = reader.getTermVector(docId, field);
+  public OffsetsEnum getOffsetsEnum(LeafReader reader, int docId, String content) throws IOException {
+    Terms tvTerms = reader.getTermVector(docId, getField());
     if (tvTerms == null) {
       return OffsetsEnum.EMPTY;
     }
 
-    LeafReader leafReader = new TermVectorLeafReader(field, tvTerms);
-    docId = 0;
-
-    return createOffsetsEnumFromReader(leafReader, docId);
+    LeafReader singleDocReader = new TermVectorLeafReader(getField(), tvTerms);
+    return createOffsetsEnumFromReader(
+        new OverlaySingleDocTermsLeafReader(
+            reader,
+            singleDocReader,
+            getField(),
+            docId),
+        docId);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b19ae942/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/TokenStreamOffsetStrategy.java
----------------------------------------------------------------------
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 b81e63e..677ee4b 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
@@ -22,7 +22,7 @@ import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.automaton.Automata;
 import org.apache.lucene.util.automaton.CharacterRunAutomaton;
@@ -36,9 +36,17 @@ public class TokenStreamOffsetStrategy extends AnalysisOffsetStrategy {
 
   private static final BytesRef[] ZERO_LEN_BYTES_REF_ARRAY = new BytesRef[0];
 
-  public TokenStreamOffsetStrategy(String field, BytesRef[] terms, PhraseHelper phraseHelper, CharacterRunAutomaton[] automata, Analyzer indexAnalyzer) {
-    super(field, ZERO_LEN_BYTES_REF_ARRAY, phraseHelper, convertTermsToAutomata(terms, automata), indexAnalyzer);
-    assert phraseHelper.hasPositionSensitivity() == false;
+  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);
+    assert components.getPhraseHelper().hasPositionSensitivity() == false;
   }
 
   private static CharacterRunAutomaton[] convertTermsToAutomata(BytesRef[] terms, CharacterRunAutomaton[] automata) {
@@ -58,8 +66,8 @@ public class TokenStreamOffsetStrategy extends AnalysisOffsetStrategy {
   }
 
   @Override
-  public OffsetsEnum getOffsetsEnum(IndexReader reader, int docId, String content) throws IOException {
-    return new TokenStreamOffsetsEnum(tokenStream(content), automata);
+  public OffsetsEnum getOffsetsEnum(LeafReader reader, int docId, String content) throws IOException {
+    return new TokenStreamOffsetsEnum(tokenStream(content), components.getAutomata());
   }
 
   private static class TokenStreamOffsetsEnum extends OffsetsEnum {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b19ae942/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/UHComponents.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..eed1e9c
--- /dev/null
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/UHComponents.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search.uhighlight;
+
+import java.util.Set;
+import java.util.function.Predicate;
+
+import org.apache.lucene.search.Query;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.automaton.CharacterRunAutomaton;
+
+/**
+ * A parameter object to hold the components a {@link FieldOffsetStrategy} needs.
+ *
+ * @lucene.internal
+ */
+public class UHComponents {
+  private final String field;
+  private final Predicate<String> fieldMatcher;
+  private final Query query;
+  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 Set<UnifiedHighlighter.HighlightFlag> highlightFlags;
+
+  public UHComponents(String field, Predicate<String> fieldMatcher, Query query,
+                      BytesRef[] terms, PhraseHelper phraseHelper, CharacterRunAutomaton[] automata,
+                      Set<UnifiedHighlighter.HighlightFlag> highlightFlags) {
+    this.field = field;
+    this.fieldMatcher = fieldMatcher;
+    this.query = query;
+    this.terms = terms;
+    this.phraseHelper = phraseHelper;
+    this.automata = automata;
+    this.highlightFlags = highlightFlags;
+  }
+
+  public String getField() {
+    return field;
+  }
+
+  public Predicate<String> getFieldMatcher() {
+    return fieldMatcher;
+  }
+
+  public Query getQuery() {
+    return query;
+  }
+
+  public BytesRef[] getTerms() {
+    return terms;
+  }
+
+  public PhraseHelper getPhraseHelper() {
+    return phraseHelper;
+  }
+
+  public CharacterRunAutomaton[] getAutomata() {
+    return automata;
+  }
+
+  public Set<UnifiedHighlighter.HighlightFlag> getHighlightFlags() {
+    return highlightFlags;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b19ae942/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/UnifiedHighlighter.java
----------------------------------------------------------------------
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 cfc918f..e1ece50 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
@@ -48,6 +48,7 @@ import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.index.MultiReader;
+import org.apache.lucene.index.ReaderUtil;
 import org.apache.lucene.index.StoredFieldVisitor;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.DocIdSetIterator;
@@ -623,11 +624,20 @@ public class UnifiedHighlighter {
                   && indexReaderWithTermVecCache != null)
                   ? indexReaderWithTermVecCache
                   : searcher.getIndexReader();
+          final LeafReader leafReader;
+          if (indexReader instanceof LeafReader) {
+            leafReader = (LeafReader) indexReader;
+          } else {
+            List<LeafReaderContext> leaves = indexReader.leaves();
+            LeafReaderContext leafReaderContext = leaves.get(ReaderUtil.subIndex(docId, leaves));
+            leafReader = leafReaderContext.reader();
+            docId -= leafReaderContext.docBase; // adjust 'doc' to be within this leaf reader
+          }
           int docInIndex = docInIndexes[docIdx];//original input order
           assert resultByDocIn[docInIndex] == null;
           resultByDocIn[docInIndex] =
               fieldHighlighter
-                  .highlightFieldForDoc(indexReader, docId, content.toString());
+                  .highlightFieldForDoc(leafReader, docId, content.toString());
         }
 
       }
@@ -742,13 +752,15 @@ public class UnifiedHighlighter {
   }
 
   protected FieldHighlighter getFieldHighlighter(String field, Query query, Set<Term> allTerms, int maxPassages) {
-    BytesRef[] terms = filterExtractedTerms(getFieldMatcher(field), 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);
     OffsetSource offsetSource = getOptimizedOffsetSource(field, terms, phraseHelper, automata);
+    UHComponents components = new UHComponents(field, fieldMatcher, query, terms, phraseHelper, automata, highlightFlags);
     return new FieldHighlighter(field,
-        getOffsetStrategy(offsetSource, field, terms, phraseHelper, automata, highlightFlags),
+        getOffsetStrategy(offsetSource, components),
         new SplittingBreakIterator(getBreakIterator(field), UnifiedHighlighter.MULTIVAL_SEP_CHAR),
         getScorer(field),
         maxPassages,
@@ -782,16 +794,30 @@ public class UnifiedHighlighter {
   }
 
   protected PhraseHelper getPhraseHelper(String field, Query query, Set<HighlightFlag> highlightFlags) {
+    boolean useWeightMatchesIter = highlightFlags.contains(HighlightFlag.WEIGHT_MATCHES);
+    if (useWeightMatchesIter) {
+      return PhraseHelper.NONE; // will be handled by Weight.matches which always considers phrases
+    }
     boolean highlightPhrasesStrictly = highlightFlags.contains(HighlightFlag.PHRASES);
     boolean handleMultiTermQuery = highlightFlags.contains(HighlightFlag.MULTI_TERM_QUERY);
     return highlightPhrasesStrictly ?
         new PhraseHelper(query, field, getFieldMatcher(field),
-            this::requiresRewrite, this::preSpanQueryRewrite, !handleMultiTermQuery) : PhraseHelper.NONE;
+            this::requiresRewrite,
+            this::preSpanQueryRewrite,
+            !handleMultiTermQuery
+        )
+        : PhraseHelper.NONE;
   }
 
   protected CharacterRunAutomaton[] getAutomata(String field, Query query, Set<HighlightFlag> highlightFlags) {
+    // do we "eagerly" look in span queries for automata here, or do we not and let PhraseHelper handle those?
+    // if don't highlight phrases strictly,
+    final boolean lookInSpan =
+        !highlightFlags.contains(HighlightFlag.PHRASES) // no PhraseHelper
+        || highlightFlags.contains(HighlightFlag.WEIGHT_MATCHES); // Weight.Matches will find all
+
     return highlightFlags.contains(HighlightFlag.MULTI_TERM_QUERY)
-        ? MultiTermHighlighting.extractAutomata(query, getFieldMatcher(field), !highlightFlags.contains(HighlightFlag.PHRASES), this::preMultiTermQueryRewrite)
+        ? MultiTermHighlighting.extractAutomata(query, getFieldMatcher(field), lookInSpan, this::preMultiTermQueryRewrite)
         : ZERO_LEN_AUTOMATA_ARRAY;
   }
 
@@ -829,27 +855,25 @@ public class UnifiedHighlighter {
     return offsetSource;
   }
 
-  protected FieldOffsetStrategy getOffsetStrategy(OffsetSource offsetSource, String field, BytesRef[] terms,
-                                                  PhraseHelper phraseHelper, CharacterRunAutomaton[] automata,
-                                                  Set<HighlightFlag> highlightFlags) {
+  protected FieldOffsetStrategy getOffsetStrategy(OffsetSource offsetSource, UHComponents components) {
     switch (offsetSource) {
       case ANALYSIS:
-        if (!phraseHelper.hasPositionSensitivity() &&
-            !highlightFlags.contains(HighlightFlag.PASSAGE_RELEVANCY_OVER_SPEED)) {
+        if (!components.getPhraseHelper().hasPositionSensitivity() &&
+            !components.getHighlightFlags().contains(HighlightFlag.PASSAGE_RELEVANCY_OVER_SPEED) &&
+            !components.getHighlightFlags().contains(HighlightFlag.WEIGHT_MATCHES)) {
           //skip using a memory index since it's pure term filtering
-          return new TokenStreamOffsetStrategy(field, terms, phraseHelper, automata, getIndexAnalyzer());
+          return new TokenStreamOffsetStrategy(components, getIndexAnalyzer());
         } else {
-          return new MemoryIndexOffsetStrategy(field, getFieldMatcher(field), terms, phraseHelper, automata, getIndexAnalyzer(),
-              this::preMultiTermQueryRewrite);
+          return new MemoryIndexOffsetStrategy(components, getIndexAnalyzer(), this::preMultiTermQueryRewrite);
         }
       case NONE_NEEDED:
         return NoOpOffsetStrategy.INSTANCE;
       case TERM_VECTORS:
-        return new TermVectorOffsetStrategy(field, terms, phraseHelper, automata);
+        return new TermVectorOffsetStrategy(components);
       case POSTINGS:
-        return new PostingsOffsetStrategy(field, terms, phraseHelper, automata);
+        return new PostingsOffsetStrategy(components);
       case POSTINGS_WITH_TERM_VECTORS:
-        return new PostingsWithTermVectorsOffsetStrategy(field, terms, phraseHelper, automata);
+        return new PostingsWithTermVectorsOffsetStrategy(components);
       default:
         throw new IllegalArgumentException("Unrecognized offset source " + offsetSource);
     }
@@ -1088,10 +1112,23 @@ public class UnifiedHighlighter {
    * Flags for controlling highlighting behavior.
    */
   public enum HighlightFlag {
+    /** @see UnifiedHighlighter#setHighlightPhrasesStrictly(boolean) */
     PHRASES,
+
+    /** @see UnifiedHighlighter#setHandleMultiTermQuery(boolean) */
     MULTI_TERM_QUERY,
-    PASSAGE_RELEVANCY_OVER_SPEED
-    // TODO: ignoreQueryFields
+
+    /** Passage relevancy is more important than speed.  True by default. */
+    PASSAGE_RELEVANCY_OVER_SPEED,
+
+    /**
+     * Internally use the {@link Weight#matches(LeafReaderContext, int)} API for highlighting.
+     * It's more accurate to the query, though might not calculate passage relevancy as well.
+     * Use of this flag requires {@link #MULTI_TERM_QUERY} and {@link #PHRASES}.
+     * {@link #PASSAGE_RELEVANCY_OVER_SPEED} will be ignored.  False by default.
+     */
+    WEIGHT_MATCHES
+
     // TODO: useQueryBoosts
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b19ae942/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighter.java
----------------------------------------------------------------------
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 fa6f286..3e2cc2e 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
@@ -86,33 +86,39 @@ public class TestUnifiedHighlighter extends LuceneTestCase {
   }
 
   static UnifiedHighlighter randomUnifiedHighlighter(IndexSearcher searcher, Analyzer indexAnalyzer) {
-    return randomUnifiedHighlighter(searcher, indexAnalyzer, EnumSet.noneOf(HighlightFlag.class));
+    return randomUnifiedHighlighter(searcher, indexAnalyzer, EnumSet.noneOf(HighlightFlag.class), null);
   }
 
   static UnifiedHighlighter randomUnifiedHighlighter(IndexSearcher searcher, Analyzer indexAnalyzer,
-                                                             EnumSet<HighlightFlag> mandatoryFlags) {
-    if (random().nextBoolean()) {
-      return new UnifiedHighlighter(searcher, indexAnalyzer);
-    } else {
-      final UnifiedHighlighter uh = new UnifiedHighlighter(searcher, indexAnalyzer) {
-        @Override
-        protected Set<HighlightFlag> getFlags(String field) {
-          final EnumSet<HighlightFlag> result = EnumSet.copyOf(mandatoryFlags);
-          int r = random().nextInt();
-          for (HighlightFlag highlightFlag : HighlightFlag.values()) {
-            if (((1 << highlightFlag.ordinal()) & r) == 0) {
-              result.add(highlightFlag);
-            }
+                                                     EnumSet<HighlightFlag> mandatoryFlags, Boolean requireFieldMatch) {
+    final UnifiedHighlighter uh = new UnifiedHighlighter(searcher, indexAnalyzer) {
+      Set<HighlightFlag> flags; // consistently random set of flags for this test run
+      @Override
+      protected Set<HighlightFlag> getFlags(String field) {
+        if (flags != null) {
+          return flags;
+        }
+        final EnumSet<HighlightFlag> result = EnumSet.copyOf(mandatoryFlags);
+        int r = random().nextInt();
+        for (HighlightFlag highlightFlag : HighlightFlag.values()) {
+          if (((1 << highlightFlag.ordinal()) & r) == 0) {
+            result.add(highlightFlag);
           }
-          return result;
         }
-      };
-      uh.setCacheFieldValCharsThreshold(random().nextInt(100));
-      if (random().nextBoolean()) {
-        uh.setFieldMatcher(f -> true); // requireFieldMatch==false
+        if (result.contains(HighlightFlag.WEIGHT_MATCHES)) {
+          // these two are required for WEIGHT_MATCHES
+          result.add(HighlightFlag.MULTI_TERM_QUERY);
+          result.add(HighlightFlag.PHRASES);
+        }
+        return flags = result;
       }
-      return uh;
+    };
+    uh.setCacheFieldValCharsThreshold(random().nextInt(100));
+    if (requireFieldMatch == Boolean.FALSE || (requireFieldMatch == null && random().nextBoolean())) {
+      uh.setFieldMatcher(f -> true); // requireFieldMatch==false
     }
+    return uh;
+
   }
 
   //
@@ -420,7 +426,11 @@ public class TestUnifiedHighlighter extends LuceneTestCase {
     highlighter.setHighlightPhrasesStrictly(false);
     String snippets[] = highlighter.highlight("body", query, topDocs, 2);
     assertEquals(1, snippets.length);
-    assertTrue(snippets[0].contains("<b>Buddhist</b> <b>origins</b>"));
+    if (highlighter.getFlags("body").containsAll(EnumSet.of(HighlightFlag.WEIGHT_MATCHES, HighlightFlag.PHRASES))) {
+      assertTrue(snippets[0], snippets[0].contains("<b>Buddhist origins</b>"));
+    } else {
+      assertTrue(snippets[0], snippets[0].contains("<b>Buddhist</b> <b>origins</b>"));
+    }
     ir.close();
   }
 
@@ -1113,7 +1123,7 @@ public class TestUnifiedHighlighter extends LuceneTestCase {
         return (qf) -> true;
       }
     };
-    UnifiedHighlighter highlighterFieldMatch = randomUnifiedHighlighter(searcher, indexAnalyzer, EnumSet.of(HighlightFlag.MULTI_TERM_QUERY));
+    UnifiedHighlighter highlighterFieldMatch = randomUnifiedHighlighter(searcher, indexAnalyzer, EnumSet.of(HighlightFlag.MULTI_TERM_QUERY), null);
     highlighterFieldMatch.setFieldMatcher(null);//default
     BooleanQuery.Builder queryBuilder =
         new BooleanQuery.Builder()
@@ -1187,6 +1197,23 @@ public class TestUnifiedHighlighter extends LuceneTestCase {
     ir.close();
   }
 
+  public void testMatchesSlopBug() throws IOException {
+    IndexReader ir = indexSomeFields();
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    Query query = new PhraseQuery(2, "title", "this", "is", "the", "field");
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(1, topDocs.totalHits.value);
+    String[] snippets = highlighter.highlight("title", query, topDocs, 10);
+    assertEquals(1, snippets.length);
+    if (highlighter.getFlags("title").contains(HighlightFlag.WEIGHT_MATCHES)) {
+      assertEquals("<b>This is the title field</b>.", snippets[0]);
+    } else {
+      assertEquals("<b>This</b> <b>is</b> <b>the</b> title <b>field</b>.", snippets[0]);
+    }
+    ir.close();
+  }
+
   public void testFieldMatcherPhraseQuery() throws Exception {
     IndexReader ir = indexSomeFields();
     IndexSearcher searcher = newSearcher(ir);
@@ -1197,7 +1224,7 @@ public class TestUnifiedHighlighter extends LuceneTestCase {
         return (qf) -> true;
       }
     };
-    UnifiedHighlighter highlighterFieldMatch = randomUnifiedHighlighter(searcher, indexAnalyzer, EnumSet.of(HighlightFlag.PHRASES));
+    UnifiedHighlighter highlighterFieldMatch = randomUnifiedHighlighter(searcher, indexAnalyzer, EnumSet.of(HighlightFlag.PHRASES, HighlightFlag.MULTI_TERM_QUERY), null);
     highlighterFieldMatch.setFieldMatcher(null);//default
     BooleanQuery.Builder queryBuilder =
         new BooleanQuery.Builder()
@@ -1214,16 +1241,28 @@ public class TestUnifiedHighlighter extends LuceneTestCase {
       assertEquals(1, topDocs.totalHits.value);
       String[] snippets = highlighterNoFieldMatch.highlight("title", query, topDocs, 10);
       assertEquals(1, snippets.length);
-      assertEquals("<b>This</b> <b>is</b> <b>the</b> <b>title</b> <b>field</b>.", snippets[0]);
+      if (highlighterNoFieldMatch.getFlags("title").contains(HighlightFlag.WEIGHT_MATCHES)) {
+        assertEquals("<b>This is the title field</b>.", snippets[0]);
+      } else {
+        assertEquals("<b>This</b> <b>is</b> <b>the</b> <b>title</b> <b>field</b>.", snippets[0]);
+      }
 
       snippets = highlighterFieldMatch.highlight("title", query, topDocs, 10);
       assertEquals(1, snippets.length);
-      assertEquals("<b>This</b> <b>is</b> <b>the</b> <b>title</b> field.", snippets[0]);
+      if (highlighterFieldMatch.getFlags("title").contains(HighlightFlag.WEIGHT_MATCHES)) {
+        assertEquals("<b>This is the title</b> field.", snippets[0]);
+      } else {
+        assertEquals("<b>This</b> <b>is</b> <b>the</b> <b>title</b> field.", snippets[0]);
+      }
 
       highlighterFieldMatch.setFieldMatcher((fq) -> "text".equals(fq));
       snippets = highlighterFieldMatch.highlight("title", query, topDocs, 10);
       assertEquals(1, snippets.length);
-      assertEquals("<b>This</b> <b>is</b> the title field.", snippets[0]);
+      if (highlighterFieldMatch.getFlags("title").contains(HighlightFlag.WEIGHT_MATCHES)) {
+        assertEquals("<b>This is</b> the title field.", snippets[0]);
+      } else {
+        assertEquals("<b>This</b> <b>is</b> the title field.", snippets[0]);
+      }
       highlighterFieldMatch.setFieldMatcher(null);
     }
 
@@ -1233,11 +1272,21 @@ public class TestUnifiedHighlighter extends LuceneTestCase {
       assertEquals(1, topDocs.totalHits.value);
       String[] snippets = highlighterNoFieldMatch.highlight("text", query, topDocs, 10);
       assertEquals(1, snippets.length);
-      assertEquals("<b>This</b> <b>is</b> <b>the</b> <b>text</b> <b>field</b>. <b>You</b> <b>can</b> <b>put</b> some <b>text</b> if you want.", snippets[0]);
+      if (highlighterNoFieldMatch.getFlags("text").contains(HighlightFlag.WEIGHT_MATCHES)) {
+        assertEquals("<b>This is the text field</b>. <b>You can put some text</b> if you want.", snippets[0]);
+      } else {
+        assertEquals("<b>This</b> <b>is</b> <b>the</b> <b>text</b> <b>field</b>. <b>You</b> <b>can</b> <b>put</b> some <b>text</b> if you want.", snippets[0]);
+      }
 
       snippets = highlighterFieldMatch.highlight("text", query, topDocs, 10);
       assertEquals(1, snippets.length);
-      assertEquals("<b>This</b> <b>is</b> the <b>text</b> field. <b>You</b> <b>can</b> <b>put</b> some <b>text</b> if you want.", snippets[0]);
+      if (highlighterFieldMatch.getFlags("text").contains(HighlightFlag.WEIGHT_MATCHES)) {
+        assertEquals("<b>This is</b> the text field. <b>You can put some text</b> if you want.", snippets[0]);
+      } else {
+        // note: odd that the first "text" is highlighted.  Apparently WSTE converts PhraseQuery to a SpanNearQuery with
+        //   with inorder=false when non-0 slop.  Probably a bug.
+        assertEquals("<b>This</b> <b>is</b> the <b>text</b> field. <b>You</b> <b>can</b> <b>put</b> some <b>text</b> if you want.", snippets[0]);
+      }
 
       highlighterFieldMatch.setFieldMatcher((fq) -> "title".equals(fq));
       snippets = highlighterFieldMatch.highlight("text", query, topDocs, 10);
@@ -1252,19 +1301,60 @@ public class TestUnifiedHighlighter extends LuceneTestCase {
       assertEquals(1, topDocs.totalHits.value);
       String[] snippets = highlighterNoFieldMatch.highlight("category", query, topDocs, 10);
       assertEquals(1, snippets.length);
-      assertEquals("<b>This</b> <b>is</b> <b>the</b> category <b>field</b>.", snippets[0]);
+      if (highlighterNoFieldMatch.getFlags("category").contains(HighlightFlag.WEIGHT_MATCHES)) {
+        assertEquals("<b>This is the category field</b>.", snippets[0]);
+      } else {
+        assertEquals("<b>This</b> <b>is</b> <b>the</b> category <b>field</b>.", snippets[0]);
+      }
 
       snippets = highlighterFieldMatch.highlight("category", query, topDocs, 10);
       assertEquals(1, snippets.length);
-      assertEquals("<b>This</b> <b>is</b> <b>the</b> category <b>field</b>.", snippets[0]);
-
+      if (highlighterFieldMatch.getFlags("category").contains(HighlightFlag.WEIGHT_MATCHES)) {
+        assertEquals("<b>This is the category field</b>.", snippets[0]);
+      } else {
+        assertEquals("<b>This</b> <b>is</b> <b>the</b> category <b>field</b>.", snippets[0]);
+      }
 
       highlighterFieldMatch.setFieldMatcher((fq) -> "text".equals(fq));
       snippets = highlighterFieldMatch.highlight("category", query, topDocs, 10);
       assertEquals(1, snippets.length);
-      assertEquals("<b>This</b> <b>is</b> the category field.", snippets[0]);
+      if (highlighterFieldMatch.getFlags("category").contains(HighlightFlag.WEIGHT_MATCHES)) {
+        assertEquals("<b>This is</b> the category field.", snippets[0]);
+      } else {
+        assertEquals("<b>This</b> <b>is</b> the category field.", snippets[0]);
+      }
       highlighterFieldMatch.setFieldMatcher(null);
     }
     ir.close();
   }
+
+  // LUCENE-7909
+  public void testNestedBooleanQueryAccuracy() throws IOException {
+    IndexReader ir = indexSomeFields();
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = randomUnifiedHighlighter(searcher, indexAnalyzer,
+        EnumSet.of(HighlightFlag.WEIGHT_MATCHES), true);
+
+    // This query contains an inner Boolean of two MUST clauses (== "AND").  Only one of them is
+    //  actually in the data, the other is not.  We should highlight neither.  We can highlight the outer
+    //  SHOULD clauses though.
+    Query query = new BooleanQuery.Builder()
+        .add(new PhraseQuery("title", "title", "field"), BooleanClause.Occur.SHOULD)
+        .add(new BooleanQuery.Builder()
+            .add(new TermQuery(new Term("category", "category")), BooleanClause.Occur.MUST)
+            .add(new TermQuery(new Term("category", "nonexistent")), BooleanClause.Occur.MUST)
+            .build(), BooleanClause.Occur.SHOULD)
+        .build();
+
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+
+    String[] snippets = highlighter.highlight("title", query, topDocs);
+    assertArrayEquals(new String[]{"This is the <b>title field</b>."}, snippets);
+
+    // no highlights, not of "category" since "nonexistent" wasn't there
+    snippets = highlighter.highlight("category", query, topDocs);
+    assertArrayEquals(new String[]{"This is the category field."}, snippets);
+
+    ir.close();
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b19ae942/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 2f97ad4..44d6f7b 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
@@ -154,7 +154,7 @@ public class TestUnifiedHighlighterMTQ extends LuceneTestCase {
 
   private UnifiedHighlighter randomUnifiedHighlighter(IndexSearcher searcher, Analyzer indexAnalyzer) {
     return TestUnifiedHighlighter.randomUnifiedHighlighter(searcher, indexAnalyzer,
-        EnumSet.of(HighlightFlag.MULTI_TERM_QUERY));
+        EnumSet.of(HighlightFlag.MULTI_TERM_QUERY), null);
   }
 
   public void testOnePrefix() throws Exception {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b19ae942/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterRanking.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterRanking.java b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterRanking.java
index ac7e9bc..2ddd2eb 100644
--- a/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterRanking.java
+++ b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterRanking.java
@@ -17,8 +17,10 @@
 package org.apache.lucene.search.uhighlight;
 
 import java.io.IOException;
+import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.Random;
+import java.util.Set;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
@@ -275,6 +277,15 @@ public class TestUnifiedHighlighterRanking extends LuceneTestCase {
     IndexSearcher searcher = newSearcher(ir);
     UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer) {
       @Override
+      protected Set<HighlightFlag> getFlags(String field) {
+        if (random().nextBoolean()) {
+          return EnumSet.of(HighlightFlag.MULTI_TERM_QUERY, HighlightFlag.PHRASES, HighlightFlag.WEIGHT_MATCHES);
+        } else {
+          return super.getFlags(field);
+        }
+      }
+
+      @Override
       protected PassageScorer getScorer(String field) {
         return new PassageScorer(1.2f, 0, 87);
       }
@@ -315,6 +326,15 @@ public class TestUnifiedHighlighterRanking extends LuceneTestCase {
     IndexSearcher searcher = newSearcher(ir);
     UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer) {
       @Override
+      protected Set<HighlightFlag> getFlags(String field) {
+        if (random().nextBoolean()) {
+          return EnumSet.of(HighlightFlag.MULTI_TERM_QUERY, HighlightFlag.PHRASES, HighlightFlag.WEIGHT_MATCHES);
+        } else {
+          return super.getFlags(field);
+        }
+      }
+
+      @Override
       protected PassageScorer getScorer(String field) {
         return new PassageScorer(0, 0.75f, 87);
       }