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 2016/10/04 20:12:11 UTC

[6/6] lucene-solr:master: LUCENE-7438: New UnifiedHighlighter

LUCENE-7438: New UnifiedHighlighter


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

Branch: refs/heads/master
Commit: 722e82712435ecf46c9868137d885484152f749b
Parents: cf18c45
Author: David Smiley <ds...@apache.org>
Authored: Tue Oct 4 16:11:43 2016 -0400
Committer: David Smiley <ds...@apache.org>
Committed: Tue Oct 4 16:11:43 2016 -0400

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |    6 +
 .../uhighlight/AnalysisOffsetStrategy.java      |  190 ++++
 .../uhighlight/DefaultPassageFormatter.java     |  138 +++
 .../search/uhighlight/FieldHighlighter.java     |  276 +++++
 .../search/uhighlight/FieldOffsetStrategy.java  |  122 +++
 .../uhighlight/MultiTermHighlighting.java       |  379 +++++++
 .../uhighlight/MultiValueTokenStream.java       |  148 +++
 .../search/uhighlight/NoOpOffsetStrategy.java   |   50 +
 .../lucene/search/uhighlight/OffsetsEnum.java   |   97 ++
 .../lucene/search/uhighlight/Passage.java       |  161 +++
 .../search/uhighlight/PassageFormatter.java     |   40 +
 .../lucene/search/uhighlight/PassageScorer.java |  113 ++
 .../lucene/search/uhighlight/PhraseHelper.java  |  581 ++++++++++
 .../uhighlight/PostingsOffsetStrategy.java      |   61 ++
 .../PostingsWithTermVectorsOffsetStrategy.java  |   71 ++
 .../uhighlight/SplittingBreakIterator.java      |  244 +++++
 .../TermVectorFilteredLeafReader.java           |  130 +++
 .../uhighlight/TermVectorOffsetStrategy.java    |   68 ++
 .../uhighlight/TokenStreamFromTermVector.java   |  395 +++++++
 .../search/uhighlight/UnifiedHighlighter.java   | 1021 ++++++++++++++++++
 .../lucene/search/uhighlight/package-info.java  |   22 +
 .../lucene/search/uhighlight/CambridgeMA.utf8   |    1 +
 .../uhighlight/TestSplittingBreakIterator.java  |  192 ++++
 .../uhighlight/TestUnifiedHighlighter.java      |  962 +++++++++++++++++
 .../uhighlight/TestUnifiedHighlighterMTQ.java   |  936 ++++++++++++++++
 .../TestUnifiedHighlighterRanking.java          |  339 ++++++
 .../TestUnifiedHighlighterReanalysis.java       |   74 ++
 .../TestUnifiedHighlighterStrictPhrases.java    |  404 +++++++
 .../TestUnifiedHighlighterTermVec.java          |  182 ++++
 .../lucene/search/uhighlight/UHTestHelper.java  |   69 ++
 .../TestUnifiedHighlighterExtensibility.java    |  182 ++++
 31 files changed, 7654 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/722e8271/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 32f6b51..a1273d7 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -44,6 +44,12 @@ API Changes
 
 New Features
 
+* LUCENE-7438: New "UnifiedHighlighter" derivative of the PostingsHighlighter that
+  can consume offsets from postings, term vectors, or analysis.  It can highlight phrases
+  as accurately as the standard Highlighter. Light term vectors can be used with offsets
+  in postings for fast wildcard (MultiTermQuery) highlighting.
+  (David Smiley, Timothy Rodriguez)
+
 Bug Fixes
 
 * LUCENE-7472: MultiFieldQueryParser.getFieldQuery() drops queries that are

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/722e8271/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
new file mode 100644
index 0000000..553a636
--- /dev/null
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/AnalysisOffsetStrategy.java
@@ -0,0 +1,190 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+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.Terms;
+import org.apache.lucene.index.memory.MemoryIndex;
+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;
+
+
+/**
+ * Uses an {@link Analyzer} on content to get offsets. It may use a {@link MemoryIndex} too.
+ *
+ * @lucene.internal
+ */
+public class AnalysisOffsetStrategy extends FieldOffsetStrategy {
+
+  //TODO: Consider splitting this highlighter into a MemoryIndexFieldHighlighter and a TokenStreamFieldHighlighter
+  private static final BytesRef[] ZERO_LEN_BYTES_REF_ARRAY = new BytesRef[0];
+  private final Analyzer analyzer;
+  private final MemoryIndex memoryIndex;
+  private final LeafReader leafReader;
+  private final CharacterRunAutomaton preMemIndexFilterAutomaton;
+
+  public AnalysisOffsetStrategy(String field, BytesRef[] extractedTerms, PhraseHelper phraseHelper, CharacterRunAutomaton[] automata, Analyzer analyzer) {
+    super(field, extractedTerms, phraseHelper, automata);
+    this.analyzer = analyzer;
+    // Automata (Wildcards / MultiTermQuery):
+    this.automata = automata;
+
+    if (terms.length > 0 && !strictPhrases.hasPositionSensitivity()) {
+      this.automata = convertTermsToAutomata(terms, automata);
+      // clear the terms array now that we've moved them to be expressed as automata
+      terms = ZERO_LEN_BYTES_REF_ARRAY;
+    }
+
+    if (terms.length > 0 || strictPhrases.willRewrite()) { //needs MemoryIndex
+      // init MemoryIndex
+      boolean storePayloads = strictPhrases.hasPositionSensitivity(); // might be needed
+      memoryIndex = new MemoryIndex(true, storePayloads);//true==store offsets
+      leafReader = (LeafReader) memoryIndex.createSearcher().getIndexReader();
+      // preFilter for MemoryIndex
+      preMemIndexFilterAutomaton = buildCombinedAutomaton(field, terms, this.automata, strictPhrases);
+    } else {
+      memoryIndex = null;
+      leafReader = null;
+      preMemIndexFilterAutomaton = null;
+    }
+
+  }
+
+  @Override
+  public UnifiedHighlighter.OffsetSource getOffsetSource() {
+    return UnifiedHighlighter.OffsetSource.ANALYSIS;
+  }
+
+  @Override
+  public List<OffsetsEnum> getOffsetsEnums(IndexReader reader, int docId, String content) throws IOException {
+    // note: don't need LimitTokenOffsetFilter since content is already truncated to maxLength
+    TokenStream tokenStream = tokenStream(content);
+
+    if (memoryIndex != null) { // also handles automata.length > 0
+      // We use a MemoryIndex and index the tokenStream so that later we have the PostingsEnum with offsets.
+
+      // note: An *alternative* strategy is to get PostingsEnums without offsets from the main index
+      //  and then marry this up with a fake PostingsEnum backed by a TokenStream (which has the offsets) and
+      //  can use that to filter applicable tokens?  It would have the advantage of being able to exit
+      //  early and save some re-analysis.  This would be an additional method/offset-source approach
+      //  since it's still useful to highlight without any index (so we build MemoryIndex).
+
+      // note: probably unwise to re-use TermsEnum on reset mem index so we don't. But we do re-use the
+      //   leaf reader, which is a bit more top level than in the guts.
+      memoryIndex.reset();
+
+      // Filter the tokenStream to applicable terms
+      if (preMemIndexFilterAutomaton != null) {
+        tokenStream = newKeepWordFilter(tokenStream, preMemIndexFilterAutomaton);
+      }
+      memoryIndex.addField(field, tokenStream);//note: calls tokenStream.reset() & close()
+      tokenStream = null; // it's consumed; done.
+      docId = 0;
+
+      if (automata.length > 0) {
+        Terms foundTerms = leafReader.terms(field);
+        if (foundTerms == null) {
+          return Collections.emptyList(); //No offsets for this field.
+        }
+        // Un-invert for the automata. Much more compact than a CachingTokenStream
+        tokenStream = MultiTermHighlighting.uninvertAndFilterTerms(foundTerms, 0, automata, content.length());
+      }
+
+    }
+
+    return createOffsetsEnums(leafReader, docId, tokenStream);
+  }
+
+  protected TokenStream tokenStream(String content) throws IOException {
+    return MultiValueTokenStream.wrap(field, analyzer, content, UnifiedHighlighter.MULTIVAL_SEP_CHAR);
+  }
+
+  private static CharacterRunAutomaton[] convertTermsToAutomata(BytesRef[] terms, CharacterRunAutomaton[] automata) {
+    CharacterRunAutomaton[] newAutomata = new CharacterRunAutomaton[terms.length + automata.length];
+    for (int i = 0; i < terms.length; i++) {
+      newAutomata[i] = MultiTermHighlighting.makeStringMatchAutomata(terms[i]);
+    }
+    // Append existing automata (that which is used for MTQs)
+    System.arraycopy(automata, 0, newAutomata, terms.length, automata.length);
+    return newAutomata;
+  }
+
+  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?
+    return new FilteringTokenFilter(tokenStream) {
+      final CharTermAttribute charAtt = addAttribute(CharTermAttribute.class);
+
+      @Override
+      protected boolean accept() throws IOException {
+        return charRunAutomaton.run(charAtt.buffer(), 0, charAtt.length());
+      }
+    };
+  }
+
+
+  /**
+   * Build one {@link CharacterRunAutomaton} matching any term the query might match.
+   */
+  private static CharacterRunAutomaton buildCombinedAutomaton(String field, BytesRef[] terms,
+                                                              CharacterRunAutomaton[] automata,
+                                                              PhraseHelper strictPhrases) {
+    List<CharacterRunAutomaton> allAutomata = new ArrayList<>();
+    if (terms.length > 0) {
+      allAutomata.add(new CharacterRunAutomaton(Automata.makeStringUnion(Arrays.asList(terms))));
+    }
+    Collections.addAll(allAutomata, automata);
+    for (SpanQuery spanQuery : strictPhrases.getSpanQueries()) {
+      Collections.addAll(allAutomata,
+          MultiTermHighlighting.extractAutomata(spanQuery, field, true));//true==lookInSpan
+    }
+
+    if (allAutomata.size() == 1) {
+      return allAutomata.get(0);
+    }
+    //TODO it'd be nice if we could get at the underlying Automaton in CharacterRunAutomaton so that we
+    //  could union them all. But it's not exposed, and note TermRangeQuery isn't modelled as an Automaton
+    //  by MultiTermHighlighting.
+
+    // Return an aggregate CharacterRunAutomaton of others
+    return new CharacterRunAutomaton(Automata.makeEmpty()) {// the makeEmpty() is bogus; won't be used
+      @Override
+      public boolean run(char[] chars, int offset, int length) {
+        for (int i = 0; i < allAutomata.size(); i++) {// don't use foreach to avoid Iterator allocation
+          if (allAutomata.get(i).run(chars, offset, length)) {
+            return true;
+          }
+        }
+        return false;
+      }
+    };
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/722e8271/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/DefaultPassageFormatter.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/DefaultPassageFormatter.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/DefaultPassageFormatter.java
new file mode 100644
index 0000000..77612a7
--- /dev/null
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/DefaultPassageFormatter.java
@@ -0,0 +1,138 @@
+/*
+ * 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;
+
+
+/**
+ * Creates a formatted snippet from the top passages.
+ * <p>
+ * The default implementation marks the query terms as bold, and places
+ * ellipses between unconnected passages.
+ */
+public class DefaultPassageFormatter extends PassageFormatter {
+    /** text that will appear before highlighted terms */
+    protected final String preTag;
+    /** text that will appear after highlighted terms */
+    protected final String postTag;
+    /** text that will appear between two unconnected passages */
+    protected final String ellipsis;
+    /** true if we should escape for html */
+    protected final boolean escape;
+
+    /**
+     * Creates a new DefaultPassageFormatter with the default tags.
+     */
+    public DefaultPassageFormatter() {
+        this("<b>", "</b>", "... ", false);
+    }
+
+    /**
+     * Creates a new DefaultPassageFormatter with custom tags.
+     * @param preTag text which should appear before a highlighted term.
+     * @param postTag text which should appear after a highlighted term.
+     * @param ellipsis text which should be used to connect two unconnected passages.
+     * @param escape true if text should be html-escaped
+     */
+    public DefaultPassageFormatter(String preTag, String postTag, String ellipsis, boolean escape) {
+        if (preTag == null || postTag == null || ellipsis == null) {
+            throw new NullPointerException();
+        }
+        this.preTag = preTag;
+        this.postTag = postTag;
+        this.ellipsis = ellipsis;
+        this.escape = escape;
+    }
+
+    @Override
+    public String format(Passage passages[], String content) {
+        StringBuilder sb = new StringBuilder();
+        int pos = 0;
+        for (Passage passage : passages) {
+            // don't add ellipsis if its the first one, or if its connected.
+            if (passage.startOffset > pos && pos > 0) {
+                sb.append(ellipsis);
+            }
+            pos = passage.startOffset;
+            for (int i = 0; i < passage.numMatches; i++) {
+                int start = passage.matchStarts[i];
+                int end = passage.matchEnds[i];
+                // its possible to have overlapping terms
+                if (start > pos) {
+                    append(sb, content, pos, start);
+                }
+                if (end > pos) {
+                    sb.append(preTag);
+                    append(sb, content, Math.max(pos, start), end);
+                    sb.append(postTag);
+                    pos = end;
+                }
+            }
+            // its possible a "term" from the analyzer could span a sentence boundary.
+            append(sb, content, pos, Math.max(pos, passage.endOffset));
+            pos = passage.endOffset;
+        }
+        return sb.toString();
+    }
+
+    /**
+     * Appends original text to the response.
+     * @param dest resulting text, possibly transformed or encoded
+     * @param content original text content
+     * @param start index of the first character in content
+     * @param end index of the character following the last character in content
+     */
+    protected void append(StringBuilder dest, String content, int start, int end) {
+        if (escape) {
+            // note: these are the rules from owasp.org
+            for (int i = start; i < end; i++) {
+                char ch = content.charAt(i);
+                switch(ch) {
+                    case '&':
+                        dest.append("&amp;");
+                        break;
+                    case '<':
+                        dest.append("&lt;");
+                        break;
+                    case '>':
+                        dest.append("&gt;");
+                        break;
+                    case '"':
+                        dest.append("&quot;");
+                        break;
+                    case '\'':
+                        dest.append("&#x27;");
+                        break;
+                    case '/':
+                        dest.append("&#x2F;");
+                        break;
+                    default:
+                        if (ch >= 0x30 && ch <= 0x39 || ch >= 0x41 && ch <= 0x5A || ch >= 0x61 && ch <= 0x7A) {
+                            dest.append(ch);
+                        } else if (ch < 0xff) {
+                            dest.append("&#");
+                            dest.append((int)ch);
+                            dest.append(";");
+                        } else {
+                            dest.append(ch);
+                        }
+                }
+            }
+        } else {
+            dest.append(content, start, end);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/722e8271/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
new file mode 100644
index 0000000..7c8f048
--- /dev/null
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/FieldHighlighter.java
@@ -0,0 +1,276 @@
+/*
+ * 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 java.text.BreakIterator;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.PriorityQueue;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+
+/**
+ * Internal highlighter abstraction that operates on a per field basis.
+ *
+ * @lucene.internal
+ */
+public class FieldHighlighter {
+
+  protected final String field;
+  protected final FieldOffsetStrategy fieldOffsetStrategy;
+  protected final BreakIterator breakIterator; // note: stateful!
+  protected final PassageScorer passageScorer;
+  protected final int maxPassages;
+  protected final int maxNoHighlightPassages;
+  protected final PassageFormatter passageFormatter;
+
+  public FieldHighlighter(String field, FieldOffsetStrategy fieldOffsetStrategy, BreakIterator breakIterator,
+                          PassageScorer passageScorer, int maxPassages, int maxNoHighlightPassages,
+                          PassageFormatter passageFormatter) {
+    this.field = field;
+    this.fieldOffsetStrategy = fieldOffsetStrategy;
+    this.breakIterator = breakIterator;
+    this.passageScorer = passageScorer;
+    this.maxPassages = maxPassages;
+    this.maxNoHighlightPassages = maxNoHighlightPassages;
+    this.passageFormatter = passageFormatter;
+  }
+
+  public String getField() {
+    return field;
+  }
+
+  public UnifiedHighlighter.OffsetSource getOffsetSource() {
+    return fieldOffsetStrategy.getOffsetSource();
+  }
+
+  /**
+   * 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?
+    // 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
+    }
+
+    breakIterator.setText(content);
+
+    List<OffsetsEnum> offsetsEnums = fieldOffsetStrategy.getOffsetsEnums(reader, docId, content);
+
+    Passage[] passages;
+    try {
+      // Highlight the offsetsEnum list against the content to produce Passages.
+      passages = highlightOffsetsEnums(offsetsEnums);// and breakIterator & scorer
+    } finally {
+      // Ensure closeable resources get closed
+      IOUtils.close(offsetsEnums);
+    }
+
+    // Format the resulting Passages.
+    if (passages.length == 0) {
+      // no passages were returned, so ask for a default summary
+      passages = getSummaryPassagesNoHighlight(maxNoHighlightPassages == -1 ? maxPassages : maxNoHighlightPassages);
+    }
+
+    if (passages.length > 0) {
+      return passageFormatter.format(passages, content);
+    } else {
+      return null;
+    }
+  }
+
+  /**
+   * Called to summarize a document when no highlights were found.
+   * By default this just returns the first
+   * {@link #maxPassages} sentences; subclasses can override to customize.
+   * The state of {@link #breakIterator} should be at the beginning.
+   */
+  protected Passage[] getSummaryPassagesNoHighlight(int maxPassages) {
+    assert breakIterator.current() == breakIterator.first();
+
+    List<Passage> passages = new ArrayList<>(Math.min(maxPassages, 10));
+    int pos = breakIterator.current();
+    assert pos == 0;
+    while (passages.size() < maxPassages) {
+      int next = breakIterator.next();
+      if (next == BreakIterator.DONE) {
+        break;
+      }
+      Passage passage = new Passage();
+      passage.score = Float.NaN;
+      passage.startOffset = pos;
+      passage.endOffset = next;
+      passages.add(passage);
+      pos = next;
+    }
+
+    return passages.toArray(new Passage[passages.size()]);
+  }
+
+  // algorithm: treat sentence snippets as miniature documents
+  // we can intersect these with the postings lists via BreakIterator.preceding(offset),s
+  // score each sentence as norm(sentenceStartOffset) * sum(weight * tf(freq))
+  protected Passage[] highlightOffsetsEnums(List<OffsetsEnum> offsetsEnums)
+      throws IOException {
+    PassageScorer scorer = passageScorer;
+    BreakIterator breakIterator = this.breakIterator;
+    final int contentLength = breakIterator.getText().getEndIndex();
+
+    PriorityQueue<OffsetsEnum> offsetsEnumQueue = new PriorityQueue<>(offsetsEnums.size() + 1);
+    for (OffsetsEnum off : offsetsEnums) {
+      off.weight = scorer.weight(contentLength, off.postingsEnum.freq());
+      off.nextPosition(); // go to first position
+      offsetsEnumQueue.add(off);
+    }
+    offsetsEnumQueue.add(new OffsetsEnum(null, EMPTY)); // a sentinel for termination
+
+    PriorityQueue<Passage> passageQueue = new PriorityQueue<>(Math.min(64, maxPassages + 1), (left, right) -> {
+      if (left.score < right.score) {
+        return -1;
+      } else if (left.score > right.score) {
+        return 1;
+      } else {
+        return left.startOffset - right.startOffset;
+      }
+    });
+    Passage passage = new Passage(); // the current passage in-progress.  Will either get reset or added to queue.
+
+    OffsetsEnum off;
+    while ((off = offsetsEnumQueue.poll()) != null) {
+      int start = off.startOffset();
+      if (start == -1) {
+        throw new IllegalArgumentException("field '" + field + "' was indexed without offsets, cannot highlight");
+      }
+      int end = off.endOffset();
+      // LUCENE-5166: this hit would span the content limit... however more valid
+      // hits may exist (they are sorted by start). so we pretend like we never
+      // saw this term, it won't cause a passage to be added to passageQueue or anything.
+      assert EMPTY.startOffset() == Integer.MAX_VALUE;
+      if (start < contentLength && end > contentLength) {
+        continue;
+      }
+      // See if this term should be part of a new passage.
+      if (start >= passage.endOffset) {
+        if (passage.startOffset >= 0) { // true if this passage has terms; otherwise couldn't find any (yet)
+          // finalize passage
+          passage.score *= scorer.norm(passage.startOffset);
+          // new sentence: first add 'passage' to queue
+          if (passageQueue.size() == maxPassages && passage.score < passageQueue.peek().score) {
+            passage.reset(); // can't compete, just reset it
+          } else {
+            passageQueue.offer(passage);
+            if (passageQueue.size() > maxPassages) {
+              passage = passageQueue.poll();
+              passage.reset();
+            } else {
+              passage = new Passage();
+            }
+          }
+        }
+        // if we exceed limit, we are done
+        if (start >= contentLength) {
+          break;
+        }
+        // advance breakIterator
+        passage.startOffset = Math.max(breakIterator.preceding(start + 1), 0);
+        passage.endOffset = Math.min(breakIterator.following(start), contentLength);
+      }
+      // Add this term to the passage.
+      int tf = 0;
+      while (true) {
+        tf++;
+        BytesRef term = off.getTerm();// a reference; safe to refer to
+        assert term != null;
+        passage.addMatch(start, end, term);
+        // see if there are multiple occurrences of this term in this passage. If so, add them.
+        if (!off.hasMorePositions()) {
+          break; // No more in the entire text. Already removed from pq; move on
+        }
+        off.nextPosition();
+        start = off.startOffset();
+        end = off.endOffset();
+        if (start >= passage.endOffset || end > contentLength) { // it's beyond this passage
+          offsetsEnumQueue.offer(off);
+          break;
+        }
+      }
+      passage.score += off.weight * scorer.tf(tf, passage.endOffset - passage.startOffset);
+    }
+
+    Passage[] passages = passageQueue.toArray(new Passage[passageQueue.size()]);
+    for (Passage p : passages) {
+      p.sort();
+    }
+    // sort in ascending order
+    Arrays.sort(passages, (left, right) -> left.startOffset - right.startOffset);
+    return passages;
+  }
+
+  protected static final PostingsEnum EMPTY = new PostingsEnum() {
+
+    @Override
+    public int nextPosition() throws IOException {
+      return 0;
+    }
+
+    @Override
+    public int startOffset() throws IOException {
+      return Integer.MAX_VALUE;
+    }
+
+    @Override
+    public int endOffset() throws IOException {
+      return Integer.MAX_VALUE;
+    }
+
+    @Override
+    public BytesRef getPayload() throws IOException {
+      return null;
+    }
+
+    @Override
+    public int freq() throws IOException {
+      return 0;
+    }
+
+    @Override
+    public int docID() {
+      return NO_MORE_DOCS;
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      return NO_MORE_DOCS;
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      return NO_MORE_DOCS;
+    }
+
+    @Override
+    public long cost() {
+      return 0;
+    }
+  };
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/722e8271/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
new file mode 100644
index 0000000..04df31e
--- /dev/null
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/FieldOffsetStrategy.java
@@ -0,0 +1,122 @@
+/*
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.index.IndexReader;
+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.spans.Spans;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.automaton.CharacterRunAutomaton;
+
+/**
+ * Ultimately returns a list of {@link OffsetsEnum} yielding potentially highlightable words in the text.  Needs
+ * information about the query up front.
+ *
+ * @lucene.internal
+ */
+public abstract class FieldOffsetStrategy {
+
+  protected final String field;
+  protected BytesRef[] terms; // Query: free-standing terms
+  protected PhraseHelper strictPhrases; // Query: position-sensitive information TODO: rename
+  protected CharacterRunAutomaton[] automata; // Query: free-standing wildcards (multi-term query)
+
+  public FieldOffsetStrategy(String field, BytesRef[] queryTerms, PhraseHelper phraseHelper, CharacterRunAutomaton[] automata) {
+    this.field = field;
+    this.terms = queryTerms;
+    this.strictPhrases = phraseHelper;
+    this.automata = automata;
+  }
+
+  public String getField() {
+    return field;
+  }
+
+  public abstract UnifiedHighlighter.OffsetSource getOffsetSource();
+
+  /**
+   * The primary method -- return offsets for highlightable words in the specified document.
+   * IMPORTANT: remember to close them all.
+   */
+  public abstract List<OffsetsEnum> getOffsetsEnums(IndexReader reader, int docId, String content) throws IOException;
+
+  protected List<OffsetsEnum> createOffsetsEnums(LeafReader leafReader, int doc, TokenStream tokenStream) throws IOException {
+    List<OffsetsEnum> offsetsEnums = createOffsetsEnumsFromReader(leafReader, doc);
+    if (automata.length > 0) {
+      offsetsEnums.add(createOffsetsEnumFromTokenStream(doc, tokenStream));
+    }
+    return offsetsEnums;
+  }
+
+  protected List<OffsetsEnum> createOffsetsEnumsFromReader(LeafReader atomicReader, int doc) throws IOException {
+    // For strict positions, get a Map of term to Spans:
+    //    note: ScriptPhraseHelper.NONE does the right thing for these method calls
+    final Map<BytesRef, Spans> strictPhrasesTermToSpans =
+        strictPhrases.getTermToSpans(atomicReader, doc);
+    // Usually simply wraps terms in a List; but if willRewrite() then can be expanded
+    final List<BytesRef> sourceTerms =
+        strictPhrases.expandTermsIfRewrite(terms, strictPhrasesTermToSpans);
+
+    final List<OffsetsEnum> offsetsEnums = new ArrayList<>(sourceTerms.size() + 1);
+
+    Terms termsIndex = atomicReader == null || sourceTerms.isEmpty() ? null : atomicReader.terms(field);
+    if (termsIndex != null) {
+      TermsEnum termsEnum = termsIndex.iterator();//does not return null
+      for (BytesRef term : sourceTerms) {
+        if (!termsEnum.seekExact(term)) {
+          continue; // term not found
+        }
+        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");
+        }
+        if (doc != postingsEnum.advance(doc)) { // now it's positioned, although may be exhausted
+          continue;
+        }
+        postingsEnum = strictPhrases.filterPostings(term, postingsEnum, strictPhrasesTermToSpans.get(term));
+        if (postingsEnum == null) {
+          continue;// completely filtered out
+        }
+
+        offsetsEnums.add(new OffsetsEnum(term, postingsEnum));
+      }
+    }
+    return offsetsEnums;
+  }
+
+  protected OffsetsEnum createOffsetsEnumFromTokenStream(int doc, TokenStream tokenStream) throws IOException {
+    // if there are automata (MTQ), we have to initialize the "fake" enum wrapping them.
+    assert tokenStream != null;
+    // TODO Opt: we sometimes evaluate the automata twice when this TS isn't the original; can we avoid?
+    PostingsEnum mtqPostingsEnum = MultiTermHighlighting.getDocsEnum(tokenStream, automata);
+    assert mtqPostingsEnum instanceof Closeable; // FYI we propagate close() later.
+    mtqPostingsEnum.advance(doc);
+    return new OffsetsEnum(null, mtqPostingsEnum);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/722e8271/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/MultiTermHighlighting.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/MultiTermHighlighting.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/MultiTermHighlighting.java
new file mode 100644
index 0000000..9498af5
--- /dev/null
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/MultiTermHighlighting.java
@@ -0,0 +1,379 @@
+/*
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+
+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.FilterLeafReader;
+import org.apache.lucene.index.FilteredTermsEnum;
+import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.AutomatonQuery;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.ConstantScoreQuery;
+import org.apache.lucene.search.DisjunctionMaxQuery;
+import org.apache.lucene.search.FuzzyQuery;
+import org.apache.lucene.search.PrefixQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermRangeQuery;
+import org.apache.lucene.search.spans.SpanMultiTermQueryWrapper;
+import org.apache.lucene.search.spans.SpanNearQuery;
+import org.apache.lucene.search.spans.SpanNotQuery;
+import org.apache.lucene.search.spans.SpanOrQuery;
+import org.apache.lucene.search.spans.SpanPositionCheckQuery;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.CharsRef;
+import org.apache.lucene.util.CharsRefBuilder;
+import org.apache.lucene.util.UnicodeUtil;
+import org.apache.lucene.util.automaton.Automata;
+import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.CharacterRunAutomaton;
+import org.apache.lucene.util.automaton.LevenshteinAutomata;
+import org.apache.lucene.util.automaton.Operations;
+
+/**
+ * Support for highlighting multi-term queries.
+ *
+ * @lucene.internal
+ */
+class MultiTermHighlighting {
+  private MultiTermHighlighting() {
+  }
+
+  /**
+   * Extracts all MultiTermQueries for {@code field}, and returns equivalent
+   * automata that will match terms.
+   */
+  public static CharacterRunAutomaton[] extractAutomata(Query query, String field, boolean lookInSpan) {
+    List<CharacterRunAutomaton> list = new ArrayList<>();
+    if (query instanceof BooleanQuery) {
+      for (BooleanClause clause : (BooleanQuery) query) {
+        if (!clause.isProhibited()) {
+          list.addAll(Arrays.asList(extractAutomata(clause.getQuery(), field, lookInSpan)));
+        }
+      }
+    } else if (query instanceof ConstantScoreQuery) {
+      list.addAll(Arrays.asList(extractAutomata(((ConstantScoreQuery) query).getQuery(), field, lookInSpan)));
+    } else if (query instanceof DisjunctionMaxQuery) {
+      for (Query sub : ((DisjunctionMaxQuery) query).getDisjuncts()) {
+        list.addAll(Arrays.asList(extractAutomata(sub, field, lookInSpan)));
+      }
+    } else if (lookInSpan && query instanceof SpanOrQuery) {
+      for (Query sub : ((SpanOrQuery) query).getClauses()) {
+        list.addAll(Arrays.asList(extractAutomata(sub, field, lookInSpan)));
+      }
+    } else if (lookInSpan && query instanceof SpanNearQuery) {
+      for (Query sub : ((SpanNearQuery) query).getClauses()) {
+        list.addAll(Arrays.asList(extractAutomata(sub, field, lookInSpan)));
+      }
+    } else if (lookInSpan && query instanceof SpanNotQuery) {
+      list.addAll(Arrays.asList(extractAutomata(((SpanNotQuery) query).getInclude(), field, lookInSpan)));
+    } else if (lookInSpan && query instanceof SpanPositionCheckQuery) {
+      list.addAll(Arrays.asList(extractAutomata(((SpanPositionCheckQuery) query).getMatch(), field, lookInSpan)));
+    } else if (lookInSpan && query instanceof SpanMultiTermQueryWrapper) {
+      list.addAll(Arrays.asList(extractAutomata(((SpanMultiTermQueryWrapper<?>) query).getWrappedQuery(), field, lookInSpan)));
+    } else if (query instanceof AutomatonQuery) {
+      final AutomatonQuery aq = (AutomatonQuery) query;
+      if (aq.getField().equals(field)) {
+        list.add(new CharacterRunAutomaton(aq.getAutomaton()) {
+          @Override
+          public String toString() {
+            return aq.toString();
+          }
+        });
+      }
+    } else if (query instanceof PrefixQuery) {
+      final PrefixQuery pq = (PrefixQuery) query;
+      Term prefix = pq.getPrefix();
+      if (prefix.field().equals(field)) {
+        list.add(new CharacterRunAutomaton(Operations.concatenate(Automata.makeString(prefix.text()),
+            Automata.makeAnyString())) {
+          @Override
+          public String toString() {
+            return pq.toString();
+          }
+        });
+      }
+    } else if (query instanceof FuzzyQuery) {
+      final FuzzyQuery fq = (FuzzyQuery) query;
+      if (fq.getField().equals(field)) {
+        String utf16 = fq.getTerm().text();
+        int termText[] = new int[utf16.codePointCount(0, utf16.length())];
+        for (int cp, i = 0, j = 0; i < utf16.length(); i += Character.charCount(cp)) {
+          termText[j++] = cp = utf16.codePointAt(i);
+        }
+        int termLength = termText.length;
+        int prefixLength = Math.min(fq.getPrefixLength(), termLength);
+        String suffix = UnicodeUtil.newString(termText, prefixLength, termText.length - prefixLength);
+        LevenshteinAutomata builder = new LevenshteinAutomata(suffix, fq.getTranspositions());
+        String prefix = UnicodeUtil.newString(termText, 0, prefixLength);
+        Automaton automaton = builder.toAutomaton(fq.getMaxEdits(), prefix);
+        list.add(new CharacterRunAutomaton(automaton) {
+          @Override
+          public String toString() {
+            return fq.toString();
+          }
+        });
+      }
+    } else if (query instanceof TermRangeQuery) {
+      final TermRangeQuery tq = (TermRangeQuery) query;
+      if (tq.getField().equals(field)) {
+        final CharsRef lowerBound;
+        if (tq.getLowerTerm() == null) {
+          lowerBound = null;
+        } else {
+          lowerBound = new CharsRef(tq.getLowerTerm().utf8ToString());
+        }
+
+        final CharsRef upperBound;
+        if (tq.getUpperTerm() == null) {
+          upperBound = null;
+        } else {
+          upperBound = new CharsRef(tq.getUpperTerm().utf8ToString());
+        }
+
+        final boolean includeLower = tq.includesLower();
+        final boolean includeUpper = tq.includesUpper();
+        final CharsRef scratch = new CharsRef();
+
+        @SuppressWarnings("deprecation")
+        final Comparator<CharsRef> comparator = CharsRef.getUTF16SortedAsUTF8Comparator();
+
+        // this is *not* an automaton, but its very simple
+        list.add(new CharacterRunAutomaton(Automata.makeEmpty()) {
+          @Override
+          public boolean run(char[] s, int offset, int length) {
+            scratch.chars = s;
+            scratch.offset = offset;
+            scratch.length = length;
+
+            if (lowerBound != null) {
+              int cmp = comparator.compare(scratch, lowerBound);
+              if (cmp < 0 || (!includeLower && cmp == 0)) {
+                return false;
+              }
+            }
+
+            if (upperBound != null) {
+              int cmp = comparator.compare(scratch, upperBound);
+              if (cmp > 0 || (!includeUpper && cmp == 0)) {
+                return false;
+              }
+            }
+            return true;
+          }
+
+          @Override
+          public String toString() {
+            return tq.toString();
+          }
+        });
+      }
+    }
+    return list.toArray(new CharacterRunAutomaton[list.size()]);
+  }
+
+  /**
+   * Returns a "fake" DocsAndPositionsEnum over the tokenstream, returning offsets where {@code matchers}
+   * matches tokens.
+   * <p>
+   * This is solely used internally by PostingsHighlighter: <b>DO NOT USE THIS METHOD!</b>
+   */
+  public static PostingsEnum getDocsEnum(final TokenStream ts, final CharacterRunAutomaton[] matchers) throws IOException {
+    return new TokenStreamPostingsEnum(ts, matchers);
+  }
+
+  // TODO: we could use CachingWrapperFilter, (or consume twice) to allow us to have a true freq()
+  // but this would have a performance cost for likely little gain in the user experience, it
+  // would only serve to make this method less bogus.
+  // instead, we always return freq() = Integer.MAX_VALUE and let the highlighter terminate based on offset...
+  // TODO: DWS perhaps instead OffsetsEnum could become abstract and this would be an impl?
+  private static class TokenStreamPostingsEnum extends PostingsEnum implements Closeable {
+    TokenStream stream; // becomes null when closed
+    final CharacterRunAutomaton[] matchers;
+    final CharTermAttribute charTermAtt;
+    final OffsetAttribute offsetAtt;
+
+    int currentDoc = -1;
+    int currentMatch = -1;
+    int currentStartOffset = -1;
+
+    int currentEndOffset = -1;
+
+    final BytesRef matchDescriptions[];
+
+    TokenStreamPostingsEnum(TokenStream ts, CharacterRunAutomaton[] matchers) throws IOException {
+      this.stream = ts;
+      this.matchers = matchers;
+      matchDescriptions = new BytesRef[matchers.length];
+      charTermAtt = ts.addAttribute(CharTermAttribute.class);
+      offsetAtt = ts.addAttribute(OffsetAttribute.class);
+      ts.reset();
+    }
+
+    @Override
+    public int nextPosition() throws IOException {
+      if (stream != null) {
+        while (stream.incrementToken()) {
+          for (int i = 0; i < matchers.length; i++) {
+            if (matchers[i].run(charTermAtt.buffer(), 0, charTermAtt.length())) {
+              currentStartOffset = offsetAtt.startOffset();
+              currentEndOffset = offsetAtt.endOffset();
+              currentMatch = i;
+              return 0;
+            }
+          }
+        }
+        stream.end();
+        close();
+      }
+      // exhausted
+      currentStartOffset = currentEndOffset = Integer.MAX_VALUE;
+      return Integer.MAX_VALUE;
+    }
+
+    @Override
+    public int freq() throws IOException {
+      return Integer.MAX_VALUE; // lie
+    }
+
+    @Override
+    public int startOffset() throws IOException {
+      assert currentStartOffset >= 0;
+      return currentStartOffset;
+    }
+
+    @Override
+    public int endOffset() throws IOException {
+      assert currentEndOffset >= 0;
+      return currentEndOffset;
+    }
+
+    @Override
+    public BytesRef getPayload() throws IOException {
+      if (matchDescriptions[currentMatch] == null) {
+        matchDescriptions[currentMatch] = new BytesRef(matchers[currentMatch].toString());
+      }
+      return matchDescriptions[currentMatch];
+    }
+
+    @Override
+    public int docID() {
+      return currentDoc;
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      return currentDoc = target;
+    }
+
+    @Override
+    public long cost() {
+      return 0;
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (stream != null) {
+        stream.close();
+        stream = null;
+      }
+    }
+  }
+
+  /**
+   * Return a TokenStream un-inverted from the provided Terms, but filtered based on the automata. The
+   * Terms must have exactly one doc count (e.g. term vector or MemoryIndex).
+   */
+  //TODO: Alternatively, produce a list of OffsetsEnums from the Terms that match the automata.
+  public static TokenStream uninvertAndFilterTerms(Terms termsIndex,
+                                                      int doc,
+                                                      final CharacterRunAutomaton[] automata,
+                                                      int offsetLength)
+      throws IOException {
+    assert automata.length > 0;
+    //Note: if automata were plain Automaton (not CharacterRunAutomaton), we might instead use
+    // TermsEnum.intersect(compiledAutomaton).  But probably won't help due to O(N) TV impl so whatever.
+    FilterLeafReader.FilterTerms filteredTermsIndex = new FilterLeafReader.FilterTerms(termsIndex) {
+      @Override
+      public TermsEnum iterator() throws IOException {
+        return new FilteredTermsEnum(super.iterator(), false) {//false == no seek
+          CharsRefBuilder tempCharsRefBuilder = new CharsRefBuilder();//reuse only for UTF8->UTF16 call
+
+          @Override
+          protected AcceptStatus accept(BytesRef termBytesRef) throws IOException {
+            //Grab the term (in same way as BytesRef.utf8ToString() but we don't want a String obj)
+            tempCharsRefBuilder.grow(termBytesRef.length);
+            final int charLen = UnicodeUtil.UTF8toUTF16(termBytesRef, tempCharsRefBuilder.chars());
+            for (CharacterRunAutomaton runAutomaton : automata) {
+              if (runAutomaton.run(tempCharsRefBuilder.chars(), 0, charLen)) {
+                return AcceptStatus.YES;
+              }
+            }
+            return AcceptStatus.NO;
+          }
+        };
+      }
+
+      @Override
+      public long size() throws IOException {
+        return -1; // unknown
+      }
+
+      @Override
+      public long getSumTotalTermFreq() throws IOException {
+        return -1; // unknown
+      }
+
+      @Override
+      public long getSumDocFreq() throws IOException {
+        return -1; // unknown
+      }
+    };
+    float loadFactor = 1f / 64f;
+    return new TokenStreamFromTermVector(filteredTermsIndex, doc, offsetLength, loadFactor);
+  }
+
+  /**
+   * Returns a simple automata that matches the specified term.
+   */
+  public static CharacterRunAutomaton makeStringMatchAutomata(BytesRef term) {
+    String termString = term.utf8ToString();
+    return new CharacterRunAutomaton(Automata.makeString(termString)) {
+      @Override
+      public String toString() {
+        return termString;
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/722e8271/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/MultiValueTokenStream.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/MultiValueTokenStream.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/MultiValueTokenStream.java
new file mode 100644
index 0000000..4cbf754
--- /dev/null
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/MultiValueTokenStream.java
@@ -0,0 +1,148 @@
+/*
+ * 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.analysis.Analyzer;
+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;
+
+/**
+ * Wraps an {@link Analyzer} and string text that represents multiple values delimited by a specified character. This
+ * exposes a TokenStream that matches what would get indexed considering the
+ * {@link Analyzer#getPositionIncrementGap(String)}. Currently this assumes {@link Analyzer#getOffsetGap(String)} is
+ * 1; an exception will be thrown if it isn't.
+ * <br />
+ * It would be more orthogonal for this to be an Analyzer since we're wrapping an Analyzer but doing so seems like
+ * more work.  The underlying components see a Reader not a String -- and the String is easy to
+ * split up without redundant buffering.
+ *
+ * @lucene.internal
+ */
+final class MultiValueTokenStream extends TokenFilter {
+
+    private final String fieldName;
+    private final Analyzer indexAnalyzer;
+    private final String content;
+    private final char splitChar;
+
+    private final PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class);
+    private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
+
+    private int startValIdx = 0;
+    private int endValIdx;
+    private int remainingPosInc = 0;
+
+    /** note: The caller must remember to close the TokenStream eventually. */
+    static TokenStream wrap(String fieldName, Analyzer indexAnalyzer, String content, char splitChar)
+            throws IOException {
+        if (indexAnalyzer.getOffsetGap(fieldName) != 1) { // note: 1 is the default. It is RARELY changed.
+            throw new IllegalArgumentException(
+                    "offset gap of the provided analyzer should be 1 (field " + fieldName + ")");
+        }
+        // If there is no splitChar in content then we needn't wrap:
+        int splitCharIdx = content.indexOf(splitChar);
+        if (splitCharIdx == -1) {
+            return indexAnalyzer.tokenStream(fieldName, content);
+        }
+
+        TokenStream subTokenStream = indexAnalyzer.tokenStream(fieldName, content.substring(0, splitCharIdx));
+
+        return new MultiValueTokenStream(subTokenStream, fieldName, indexAnalyzer, content, splitChar, splitCharIdx);
+    }
+
+    private MultiValueTokenStream(TokenStream subTokenStream, String fieldName, Analyzer indexAnalyzer,
+                                  String content, char splitChar, int splitCharIdx) {
+        super(subTokenStream); // subTokenStream is already initialized to operate on the first value
+        this.fieldName = fieldName;
+        this.indexAnalyzer = indexAnalyzer;
+        this.content = content;
+        this.splitChar = splitChar;
+        this.endValIdx = splitCharIdx;
+    }
+
+    @Override
+    public void reset() throws IOException {
+        if (startValIdx != 0) {
+            throw new IllegalStateException("This TokenStream wasn't developed to be re-used.");
+            // ... although we could if a need for it arises.
+        }
+        super.reset();
+    }
+
+    @Override
+    public boolean incrementToken() throws IOException {
+        while (true) {
+
+            if (input.incrementToken()) {
+                // Position tracking:
+                if (remainingPosInc > 0) {//usually true first token of additional values (not first val)
+                    posIncAtt.setPositionIncrement(remainingPosInc + posIncAtt.getPositionIncrement());
+                    remainingPosInc = 0;//reset
+                }
+                // Offset tracking:
+                offsetAtt.setOffset(
+                        startValIdx + offsetAtt.startOffset(),
+                        startValIdx + offsetAtt.endOffset()
+                                         );
+                return true;
+            }
+
+            if (endValIdx == content.length()) {//no more
+                return false;
+            }
+
+            input.end(); // might adjust position increment
+            remainingPosInc += posIncAtt.getPositionIncrement();
+            input.close();
+            remainingPosInc += indexAnalyzer.getPositionIncrementGap(fieldName);
+
+            // Get new tokenStream based on next segment divided by the splitChar
+            startValIdx = endValIdx + 1;
+            endValIdx = content.indexOf(splitChar, startValIdx);
+            if (endValIdx == -1) {//EOF
+                endValIdx = content.length();
+            }
+            TokenStream tokenStream = indexAnalyzer.tokenStream(fieldName, content.substring(startValIdx, endValIdx));
+            if (tokenStream != input) {// (input is defined in TokenFilter set in the constructor)
+                // This is a grand trick we do -- knowing that the analyzer's re-use strategy is going to produce the
+                // very same tokenStream instance and thus have the same AttributeSource as this wrapping TokenStream
+                // since we used it as our input in the constructor.
+                // Were this not the case, we'd have to copy every attribute of interest since we can't alter the
+                // AttributeSource of this wrapping TokenStream post-construction (it's all private/final).
+                // If this is a problem, we could do that instead; maybe with a custom CharTermAttribute that allows
+                // us to easily set the char[] reference without literally copying char by char.
+                throw new IllegalStateException("Require TokenStream re-use.  Unsupported re-use strategy?: " +
+                                                indexAnalyzer.getReuseStrategy());
+            }
+            tokenStream.reset();
+        } // while loop to increment token of this new value
+    }
+
+    @Override
+    public void end() throws IOException {
+        super.end();
+        // Offset tracking:
+        offsetAtt.setOffset(
+                startValIdx + offsetAtt.startOffset(),
+                startValIdx + offsetAtt.endOffset());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/722e8271/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
new file mode 100644
index 0000000..7ae4de5
--- /dev/null
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/NoOpOffsetStrategy.java
@@ -0,0 +1,50 @@
+/*
+ * 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 java.util.Collections;
+import java.util.List;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.automaton.CharacterRunAutomaton;
+
+/**
+ * Never returns offsets. Used when the query would highlight nothing.
+ *
+ * @lucene.internal
+ */
+public class NoOpOffsetStrategy extends FieldOffsetStrategy {
+
+  public static final NoOpOffsetStrategy INSTANCE = new NoOpOffsetStrategy();
+
+  private NoOpOffsetStrategy() {
+    super("_ignored_", new BytesRef[0], PhraseHelper.NONE, new CharacterRunAutomaton[0]);
+  }
+
+  @Override
+  public UnifiedHighlighter.OffsetSource getOffsetSource() {
+    return UnifiedHighlighter.OffsetSource.NONE_NEEDED;
+  }
+
+  @Override
+  public List<OffsetsEnum> getOffsetsEnums(IndexReader reader, int docId, String content) throws IOException {
+    return Collections.emptyList();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/722e8271/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
new file mode 100644
index 0000000..af29ef1
--- /dev/null
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/OffsetsEnum.java
@@ -0,0 +1,97 @@
+/*
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.Objects;
+
+import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Holds the term &amp; PostingsEnum, and info for tracking the occurrences of a term within the text.
+ * It is advanced with the underlying postings and is placed in a priority queue by highlightOffsetsEnums
+ * based on the start offset.
+ *
+ * @lucene.internal
+ */
+public class OffsetsEnum implements Comparable<OffsetsEnum>, Closeable {
+  private final BytesRef term;
+  final PostingsEnum postingsEnum; // with offsets
+
+  float weight; // set once in highlightOffsetsEnums
+  private int pos = 0; // the occurrence counter of this term within the text being highlighted.
+
+  public OffsetsEnum(BytesRef term, PostingsEnum postingsEnum) throws IOException {
+    this.term = term; // can be null
+    this.postingsEnum = Objects.requireNonNull(postingsEnum);
+  }
+
+  // note: the ordering clearly changes as the postings enum advances
+  @Override
+  public int compareTo(OffsetsEnum other) {
+    try {
+      int cmp = Integer.compare(startOffset(), other.startOffset());
+      if (cmp != 0) {
+        return cmp; // vast majority of the time we return here.
+      }
+      if (this.term == null || other.term == null) {
+        if (this.term == null && other.term == null) {
+          return 0;
+        } else if (this.term == null) {
+          return 1; // put "this" (wildcard mtq enum) last
+        } else {
+          return -1;
+        }
+      }
+      return term.compareTo(other.term);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  BytesRef getTerm() throws IOException {
+    // the dp.getPayload thing is a hack -- see MultiTermHighlighting
+    return term != null ? term : postingsEnum.getPayload();
+    // We don't deepcopy() because in this hack we know we don't have to.
+  }
+
+  boolean hasMorePositions() throws IOException {
+    return pos < postingsEnum.freq();
+  }
+
+  void nextPosition() throws IOException {
+    pos++;
+    postingsEnum.nextPosition();
+  }
+
+  int startOffset() throws IOException {
+    return postingsEnum.startOffset();
+  }
+
+  int endOffset() throws IOException {
+    return postingsEnum.endOffset();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (postingsEnum instanceof Closeable) { // the one in MultiTermHighlighting is.
+      ((Closeable) postingsEnum).close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/722e8271/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
new file mode 100644
index 0000000..f4caaa0
--- /dev/null
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/Passage.java
@@ -0,0 +1,161 @@
+/*
+ * 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 org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.InPlaceMergeSorter;
+import org.apache.lucene.util.RamUsageEstimator;
+
+/**
+ * Represents a passage (typically a sentence of the document). 
+ * <p>
+ * A passage contains {@link #getNumMatches} highlights from the query,
+ * and the offsets and query terms that correspond with each match.
+ *
+ * @lucene.experimental
+ */
+public final class Passage {
+    int startOffset = -1;
+    int endOffset = -1;
+    float score = 0.0f;
+
+    int matchStarts[] = new int[8];
+    int matchEnds[] = new int[8];
+    BytesRef matchTerms[] = new BytesRef[8];
+    int numMatches = 0;
+
+    void addMatch(int startOffset, int endOffset, BytesRef term) {
+        assert startOffset >= this.startOffset && startOffset <= this.endOffset;
+        if (numMatches == matchStarts.length) {
+            int newLength = ArrayUtil.oversize(numMatches+1, RamUsageEstimator.NUM_BYTES_OBJECT_REF);
+            int newMatchStarts[] = new int[newLength];
+            int newMatchEnds[] = new int[newLength];
+            BytesRef newMatchTerms[] = new BytesRef[newLength];
+            System.arraycopy(matchStarts, 0, newMatchStarts, 0, numMatches);
+            System.arraycopy(matchEnds, 0, newMatchEnds, 0, numMatches);
+            System.arraycopy(matchTerms, 0, newMatchTerms, 0, numMatches);
+            matchStarts = newMatchStarts;
+            matchEnds = newMatchEnds;
+            matchTerms = newMatchTerms;
+        }
+        assert matchStarts.length == matchEnds.length && matchEnds.length == matchTerms.length;
+        matchStarts[numMatches] = startOffset;
+        matchEnds[numMatches] = endOffset;
+        matchTerms[numMatches] = term;
+        numMatches++;
+    }
+
+    void sort() {
+        final int starts[] = matchStarts;
+        final int ends[] = matchEnds;
+        final BytesRef terms[] = matchTerms;
+        new InPlaceMergeSorter() {
+            @Override
+            protected void swap(int i, int j) {
+                int temp = starts[i];
+                starts[i] = starts[j];
+                starts[j] = temp;
+
+                temp = ends[i];
+                ends[i] = ends[j];
+                ends[j] = temp;
+
+                BytesRef tempTerm = terms[i];
+                terms[i] = terms[j];
+                terms[j] = tempTerm;
+            }
+
+            @Override
+            protected int compare(int i, int j) {
+                return Integer.compare(starts[i], starts[j]);
+            }
+
+        }.sort(0, numMatches);
+    }
+
+    void reset() {
+        startOffset = endOffset = -1;
+        score = 0.0f;
+        numMatches = 0;
+    }
+
+    /**
+     * Start offset of this passage.
+     * @return start index (inclusive) of the passage in the
+     *         original content: always &gt;= 0.
+     */
+    public int getStartOffset() {
+        return startOffset;
+    }
+
+    /**
+     * End offset of this passage.
+     * @return end index (exclusive) of the passage in the
+     *         original content: always &gt;= {@link #getStartOffset()}
+     */
+    public int getEndOffset() {
+        return endOffset;
+    }
+
+    /**
+     * Passage's score.
+     */
+    public float getScore() {
+        return score;
+    }
+
+    /**
+     * Number of term matches available in
+     * {@link #getMatchStarts}, {@link #getMatchEnds},
+     * {@link #getMatchTerms}
+     */
+    public int getNumMatches() {
+        return numMatches;
+    }
+
+    /**
+     * Start offsets of the term matches, in increasing order.
+     * <p>
+     * Only {@link #getNumMatches} are valid. Note that these
+     * offsets are absolute (not relative to {@link #getStartOffset()}).
+     */
+    public int[] getMatchStarts() {
+        return matchStarts;
+    }
+
+    /**
+     * End offsets of the term matches, corresponding with {@link #getMatchStarts}.
+     * <p>
+     * Only {@link #getNumMatches} are valid. Note that its possible that an end offset
+     * could exceed beyond the bounds of the passage ({@link #getEndOffset()}), if the
+     * Analyzer produced a term which spans a passage boundary.
+     */
+    public int[] getMatchEnds() {
+        return matchEnds;
+    }
+
+    /**
+     * BytesRef (term text) of the matches, corresponding with {@link #getMatchStarts()}.
+     * <p>
+     * Only {@link #getNumMatches()} are valid.
+     */
+    public BytesRef[] getMatchTerms() {
+        return matchTerms;
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/722e8271/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/PassageFormatter.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/PassageFormatter.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/PassageFormatter.java
new file mode 100644
index 0000000..c2de2aa
--- /dev/null
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/PassageFormatter.java
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * Creates a formatted snippet from the top passages.
+ *
+ * @lucene.experimental
+ */
+public abstract class PassageFormatter {
+
+  /**
+   * Formats the top <code>passages</code> from <code>content</code>
+   * into a human-readable text snippet.
+   *
+   * @param passages top-N passages for the field. Note these are sorted in
+   *                 the order that they appear in the document for convenience.
+   * @param content  content for the field.
+   * @return formatted highlight.  Note that for the
+   * non-expert APIs in {@link UnifiedHighlighter} that
+   * return String, the toString method on the Object
+   * returned by this method is used to compute the string.
+   */
+  public abstract Object format(Passage passages[], String content);
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/722e8271/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/PassageScorer.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/PassageScorer.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/PassageScorer.java
new file mode 100644
index 0000000..1ceab6f
--- /dev/null
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/PassageScorer.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;
+
+/**
+ * Ranks passages found by {@link UnifiedHighlighter}.
+ * <p>
+ * Each passage is scored as a miniature document within the document.
+ * The final score is computed as {@link #norm} * &sum; ({@link #weight} * {@link #tf}).
+ * The default implementation is {@link #norm} * BM25.
+ *
+ * @lucene.experimental
+ */
+public class PassageScorer {
+
+  // TODO: this formula is completely made up. It might not provide relevant snippets!
+
+  /**
+   * BM25 k1 parameter, controls term frequency normalization
+   */
+  final float k1;
+  /**
+   * BM25 b parameter, controls length normalization.
+   */
+  final float b;
+  /**
+   * A pivot used for length normalization.
+   */
+  final float pivot;
+
+  /**
+   * Creates PassageScorer with these default values:
+   * <ul>
+   * <li>{@code k1 = 1.2},
+   * <li>{@code b = 0.75}.
+   * <li>{@code pivot = 87}
+   * </ul>
+   */
+  public PassageScorer() {
+    // 1.2 and 0.75 are well-known bm25 defaults (but maybe not the best here) ?
+    // 87 is typical average english sentence length.
+    this(1.2f, 0.75f, 87f);
+  }
+
+  /**
+   * Creates PassageScorer with specified scoring parameters
+   *
+   * @param k1    Controls non-linear term frequency normalization (saturation).
+   * @param b     Controls to what degree passage length normalizes tf values.
+   * @param pivot Pivot value for length normalization (some rough idea of average sentence length in characters).
+   */
+  public PassageScorer(float k1, float b, float pivot) {
+    this.k1 = k1;
+    this.b = b;
+    this.pivot = pivot;
+  }
+
+  /**
+   * Computes term importance, given its in-document statistics.
+   *
+   * @param contentLength length of document in characters
+   * @param totalTermFreq number of time term occurs in document
+   * @return term importance
+   */
+  public float weight(int contentLength, int totalTermFreq) {
+    // approximate #docs from content length
+    float numDocs = 1 + contentLength / pivot;
+    // numDocs not numDocs - docFreq (ala DFR), since we approximate numDocs
+    return (k1 + 1) * (float) Math.log(1 + (numDocs + 0.5D) / (totalTermFreq + 0.5D));
+  }
+
+  /**
+   * Computes term weight, given the frequency within the passage
+   * and the passage's length.
+   *
+   * @param freq       number of occurrences of within this passage
+   * @param passageLen length of the passage in characters.
+   * @return term weight
+   */
+  public float tf(int freq, int passageLen) {
+    float norm = k1 * ((1 - b) + b * (passageLen / pivot));
+    return freq / (freq + norm);
+  }
+
+  /**
+   * Normalize a passage according to its position in the document.
+   * <p>
+   * Typically passages towards the beginning of the document are
+   * more useful for summarizing the contents.
+   * <p>
+   * The default implementation is <code>1 + 1/log(pivot + passageStart)</code>
+   *
+   * @param passageStart start offset of the passage
+   * @return a boost value multiplied into the passage's core.
+   */
+  public float norm(int passageStart) {
+    return 1 + 1 / (float) Math.log(pivot + passageStart);
+  }
+}