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:10 UTC

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/722e8271/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/PhraseHelper.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/PhraseHelper.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/PhraseHelper.java
new file mode 100644
index 0000000..5225041
--- /dev/null
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/PhraseHelper.java
@@ -0,0 +1,581 @@
+/*
+ * 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.index.*;
+import org.apache.lucene.search.*;
+import org.apache.lucene.search.highlight.WeightedSpanTerm;
+import org.apache.lucene.search.highlight.WeightedSpanTermExtractor;
+import org.apache.lucene.search.spans.*;
+import org.apache.lucene.util.BytesRef;
+
+import java.io.IOException;
+import java.util.*;
+import java.util.function.Function;
+
+/**
+ * Helps the {@link FieldOffsetStrategy} with strict position highlighting (e.g. highlight phrases correctly).
+ * This is a stateful class holding information about the query, but it can (and is) re-used across highlighting
+ * documents.  Despite this state; it's immutable after construction.  The approach taken in this class is very similar
+ * to the standard Highlighter's {@link WeightedSpanTermExtractor} which is in fact re-used here.  However, we ought to
+ * completely rewrite it to use the SpanCollector interface to collect offsets directly. We'll get better
+ * phrase accuracy.
+ *
+ * @lucene.internal
+ */
+public class PhraseHelper {
+
+  public static final PhraseHelper NONE = new PhraseHelper(new MatchAllDocsQuery(), "_ignored_",
+      spanQuery -> null, true);
+
+  //TODO it seems this ought to be a general thing on Spans?
+  private static final Comparator<? super Spans> SPANS_COMPARATOR = (o1, o2) -> {
+    int cmp = Integer.compare(o1.docID(), o2.docID());
+    if (cmp != 0) {
+      return cmp;
+    }
+    if (o1.docID() == DocIdSetIterator.NO_MORE_DOCS) {
+      return 0; // don't ask for start/end position; not sure if we can even call those methods
+    }
+    cmp = Integer.compare(o1.startPosition(), o2.startPosition());
+    if (cmp != 0) {
+      return cmp;
+    } else {
+      return Integer.compare(o1.endPosition(), o2.endPosition());
+    }
+  };
+
+  private final String fieldName; // if non-null, only look at queries/terms for this field
+  private final Set<Term> positionInsensitiveTerms; // (TermQuery terms)
+  private final Set<SpanQuery> spanQueries;
+  private final boolean willRewrite;
+
+  /**
+   * Constructor.
+   * {@code rewriteQueryPred} is an extension hook to override the default choice of
+   * {@link WeightedSpanTermExtractor#mustRewriteQuery(SpanQuery)}. By default unknown query types are rewritten,
+   * so use this to return {@link Boolean#FALSE} if you know the query doesn't need to be rewritten.
+   * {@code ignoreQueriesNeedingRewrite} effectively ignores any query clause that needs to be "rewritten", which is
+   * usually limited to just a {@link SpanMultiTermQueryWrapper} but could be other custom ones.
+   */
+  public PhraseHelper(Query query, String field, Function<SpanQuery, Boolean> rewriteQueryPred,
+               boolean ignoreQueriesNeedingRewrite) {
+    this.fieldName = field; // if null then don't require field match
+    // filter terms to those we want
+    positionInsensitiveTerms = field != null ? new FieldFilteringTermHashSet(field) : new HashSet<>();
+    // requireFieldMatch optional
+    spanQueries = new HashSet<>();
+
+    // TODO Have toSpanQuery(query) Function as an extension point for those with custom Query impls
+
+    boolean[] mustRewriteHolder = {false}; // boolean wrapped in 1-ary array so it's mutable from inner class
+
+    // For TermQueries or other position insensitive queries, collect the Terms.
+    // For other Query types, WSTE will convert to an equivalent SpanQuery.  NOT extracting position spans here.
+    new WeightedSpanTermExtractor(field) {
+      //anonymous constructor
+      {
+        setExpandMultiTermQuery(true); //necessary for mustRewriteQuery(spanQuery) to work.
+
+        try {
+          extract(query, 1f, null); // null because we won't actually extract right now; we're not collecting
+        } catch (Exception e) {
+          throw new RuntimeException(e);
+        }
+      }
+
+      @Override
+      protected boolean isQueryUnsupported(Class<? extends Query> clazz) {
+        if (clazz.isAssignableFrom(MultiTermQuery.class)) {
+          return true; //We do MTQ processing separately in MultiTermHighlighting.java
+        }
+        return true; //TODO set to false and provide a hook to customize certain queries.
+      }
+
+      @Override
+      protected void extractWeightedTerms(Map<String, WeightedSpanTerm> terms, Query query, float boost)
+          throws IOException {
+        query.createWeight(UnifiedHighlighter.EMPTY_INDEXSEARCHER, false, boost)
+            .extractTerms(positionInsensitiveTerms);
+      }
+
+      @Override
+      protected void extractWeightedSpanTerms(Map<String, WeightedSpanTerm> terms, SpanQuery spanQuery,
+                                              float boost) throws IOException {
+        if (field != null) {
+          // if this span query isn't for this field, skip it.
+          Set<String> fieldNameSet = new HashSet<>();//TODO reuse.  note: almost always size 1
+          collectSpanQueryFields(spanQuery, fieldNameSet);
+          if (!fieldNameSet.contains(field)) {
+            return;
+          }
+        }
+
+        // TODO allow users to override the answer to mustRewriteQuery
+        boolean mustRewriteQuery = mustRewriteQuery(spanQuery);
+        if (ignoreQueriesNeedingRewrite && mustRewriteQuery) {
+          return;// ignore this query
+        }
+        mustRewriteHolder[0] |= mustRewriteQuery;
+
+        spanQueries.add(spanQuery);
+      }
+
+      @Override
+      protected boolean mustRewriteQuery(SpanQuery spanQuery) {
+        Boolean rewriteQ = rewriteQueryPred.apply(spanQuery);// allow to override
+        return rewriteQ != null ? rewriteQ : super.mustRewriteQuery(spanQuery);
+      }
+    }; // calling the constructor triggered the extraction/visiting we want.  Hacky; yes.
+
+    willRewrite = mustRewriteHolder[0];
+  }
+
+  Set<SpanQuery> getSpanQueries() {
+    return spanQueries;
+  }
+
+  /**
+   * If there is no position sensitivity then use of the instance of this class can be ignored.
+   */
+  boolean hasPositionSensitivity() {
+    return spanQueries.isEmpty() == false;
+  }
+
+  /**
+   * Rewrite is needed for handling a {@link SpanMultiTermQueryWrapper} (MTQ / wildcards) or some
+   * custom things.  When true, the resulting term list will probably be different than what it was known
+   * to be initially.
+   */
+  boolean willRewrite() {
+    return willRewrite;
+  }
+
+  /**
+   * Collect a list of pre-positioned {@link Spans} for each term, given a reader that has just one document.
+   * It returns no mapping for query terms that occurs in a position insensitive way which therefore don't
+   * need to be filtered.
+   */
+  Map<BytesRef, Spans> getTermToSpans(LeafReader leafReader, int doc)
+      throws IOException {
+    if (spanQueries.isEmpty()) {
+      return Collections.emptyMap();
+    }
+    // for each SpanQuery, collect the member spans into a map.
+    Map<BytesRef, Spans> result = new HashMap<>();
+    for (SpanQuery spanQuery : spanQueries) {
+      getTermToSpans(spanQuery, leafReader.getContext(), doc, result);
+    }
+    return result;
+  }
+
+  // code extracted & refactored from WSTE.extractWeightedSpanTerms()
+  private void getTermToSpans(SpanQuery spanQuery, LeafReaderContext readerContext,
+                              int doc, Map<BytesRef, Spans> result)
+      throws IOException {
+    // note: in WSTE there was some field specific looping that seemed pointless so that isn't here.
+    final IndexSearcher searcher = new IndexSearcher(readerContext);
+    searcher.setQueryCache(null);
+    if (willRewrite) {
+      spanQuery = (SpanQuery) searcher.rewrite(spanQuery); // searcher.rewrite loops till done
+    }
+
+    // Get the underlying query terms
+
+    TreeSet<Term> termSet = new TreeSet<>(); // sorted so we can loop over results in order shortly...
+    searcher.createWeight(spanQuery, false, 1.0f).extractTerms(termSet);//needsScores==false
+
+    // Get Spans by running the query against the reader
+    // TODO it might make sense to re-use/cache the Spans instance, to advance forward between docs
+    SpanWeight spanWeight = (SpanWeight) searcher.createNormalizedWeight(spanQuery, false);
+    Spans spans = spanWeight.getSpans(readerContext, SpanWeight.Postings.POSITIONS);
+    if (spans == null) {
+      return;
+    }
+    TwoPhaseIterator twoPhaseIterator = spans.asTwoPhaseIterator();
+    if (twoPhaseIterator != null) {
+      if (twoPhaseIterator.approximation().advance(doc) != doc || !twoPhaseIterator.matches()) {
+        return;
+      }
+    } else if (spans.advance(doc) != doc) { // preposition, and return doing nothing if find none
+      return;
+    }
+
+    // Consume the Spans into a cache.  This instance is used as a source for multiple cloned copies.
+    // It's important we do this and not re-use the same original Spans instance since these will be iterated
+    // independently later on; sometimes in ways that prevents sharing the original Spans.
+    CachedSpans cachedSpansSource = new CachedSpans(spans); // consumes spans for this doc only and caches
+    spans = null;// we don't use it below
+
+    // Map terms to a Spans instance (aggregate if necessary)
+    for (final Term queryTerm : termSet) {
+      // note: we expect that at least one query term will pass these filters. This is because the collected
+      //   spanQuery list were already filtered by these conditions.
+      if (fieldName != null && fieldName.equals(queryTerm.field()) == false) {
+        continue;
+      }
+      if (positionInsensitiveTerms.contains(queryTerm)) {
+        continue;
+      }
+      // copy-constructor refers to same data (shallow) but has iteration state from the beginning
+      CachedSpans cachedSpans = new CachedSpans(cachedSpansSource);
+      // Add the span to whatever span may or may not exist
+      Spans existingSpans = result.get(queryTerm.bytes());
+      if (existingSpans != null) {
+        if (existingSpans instanceof MultiSpans) {
+          ((MultiSpans) existingSpans).addSpans(cachedSpans);
+        } else { // upgrade to MultiSpans
+          MultiSpans multiSpans = new MultiSpans();
+          multiSpans.addSpans(existingSpans);
+          multiSpans.addSpans(cachedSpans);
+          result.put(queryTerm.bytes(), multiSpans);
+        }
+      } else {
+        result.put(queryTerm.bytes(), cachedSpans);
+      }
+    }
+  }
+
+  /**
+   * Returns terms as a List, but expanded to any terms in strictPhrases' keySet if present.  That can only
+   * happen if willRewrite() is true.
+   */
+  List<BytesRef> expandTermsIfRewrite(BytesRef[] terms, Map<BytesRef, Spans> strictPhrasesTermToSpans) {
+    if (willRewrite()) {
+      Set<BytesRef> allTermSet = new LinkedHashSet<>(terms.length + strictPhrasesTermToSpans.size());
+      Collections.addAll(allTermSet, terms);//FYI already sorted; will keep order
+      if (allTermSet.addAll(strictPhrasesTermToSpans.keySet())) { // true if any were added
+        List<BytesRef> sourceTerms = Arrays.asList(allTermSet.toArray(new BytesRef[allTermSet.size()]));
+        sourceTerms.sort(Comparator.naturalOrder());
+        return sourceTerms;
+      }
+    }
+    return Arrays.asList(terms); // no rewrite; use original terms
+  }
+
+  /**
+   * Returns a filtered postings where the position must be in the given Spans.
+   * The Spans must be in a positioned state (not initial) and should not be shared between other terms.
+   * {@code postingsEnum} should be positioned at the
+   * document (the same one as the spans) but it hasn't iterated the positions yet.
+   * The Spans should be the result of a simple
+   * lookup from {@link #getTermToSpans(LeafReader, int)}, and so it could be null which could mean
+   * either it's completely filtered or that there should be no filtering; this class knows what to do.
+   * <p>
+   * Due to limitations in filtering, the {@link PostingsEnum#freq()} is un-changed even if some positions
+   * get filtered.  So when {@link PostingsEnum#nextPosition()} is called or {@code startOffset} or {@code
+   * endOffset} beyond the "real" positions, these methods returns {@link Integer#MAX_VALUE}.
+   * <p>
+   * <b>This will return null if it's completely filtered out (i.e. effectively has no postings).</b>
+   */
+  PostingsEnum filterPostings(BytesRef term, PostingsEnum postingsEnum, Spans spans)
+      throws IOException {
+    if (spans == null) {
+      if (hasPositionSensitivity() == false || positionInsensitiveTerms.contains(new Term(fieldName, term))) {
+        return postingsEnum; // no filtering
+      } else {
+        return null; // completely filtered out
+      }
+    }
+    if (postingsEnum.docID() != spans.docID()) {
+      throw new IllegalStateException("Spans & Postings doc ID misaligned or not positioned");
+    }
+
+    return new FilterLeafReader.FilterPostingsEnum(postingsEnum) {
+      // freq() is max times nextPosition can be called. We'll set this var to -1 when exhausted.
+      int remainingPositions = postingsEnum.freq();
+
+      @Override
+      public String toString() {
+        String where;
+        try {
+          where = "[" + startOffset() + ":" + endOffset() + "]";
+        } catch (IOException e) {
+          where = "[" + e + "]";
+        }
+        return "'" + term.utf8ToString() + "'@" + where + " filtered by " + spans;
+      }
+
+      @Override
+      public int nextDoc() throws IOException {
+        throw new IllegalStateException("not expected"); // don't need to implement; just used on one doc
+      }
+
+      @Override
+      public int advance(int target) throws IOException {
+        throw new IllegalStateException("not expected"); // don't need to implement; just used on one doc
+      }
+
+      @Override
+      public int nextPosition() throws IOException {
+        // loop over posting positions...
+        NEXT_POS_LOOP:
+        while (remainingPositions > 0) {
+          final int thisPos = super.nextPosition();
+          remainingPositions--;
+
+          // loop spans forward (if necessary) while the span end is behind thisPos
+          while (spans.endPosition() <= thisPos) {
+            if (spans.nextStartPosition() == Spans.NO_MORE_POSITIONS) { // advance
+              break NEXT_POS_LOOP;
+            }
+            assert spans.docID() == postingsEnum.docID();
+          }
+
+          // is this position within the span?
+          if (thisPos >= spans.startPosition()) {
+            assert thisPos < spans.endPosition(); // guaranteed by previous loop
+            return thisPos; // yay!
+          }
+          // else continue and try the next position
+        }
+        remainingPositions = -1; // signify done
+        return Integer.MAX_VALUE;
+      }
+
+      @Override
+      public int startOffset() throws IOException {
+        return remainingPositions >= 0 ? super.startOffset() : Integer.MAX_VALUE;
+      }
+
+      @Override
+      public int endOffset() throws IOException {
+        return remainingPositions >= 0 ? super.endOffset() : Integer.MAX_VALUE;
+      }
+    };
+  }
+
+  /**
+   * Simple HashSet that filters out Terms not matching a desired field on {@code add()}.
+   */
+  private static class FieldFilteringTermHashSet extends HashSet<Term> {
+    private final String field;
+
+    FieldFilteringTermHashSet(String field) {
+      this.field = field;
+    }
+
+    @Override
+    public boolean add(Term term) {
+      if (term.field().equals(field)) {
+        return super.add(term);
+      } else {
+        return false;
+      }
+    }
+  }
+
+  /**
+   * A single {@link Spans} view over multiple spans.  At least one span is mandatory, but you should probably
+   * supply more than one.  Furthermore, the given spans are expected to be positioned to a document already
+   * via a call to next or advance).
+   */  // TODO move to Lucene core as a Spans utility class?
+  static class MultiSpans extends Spans {
+    final PriorityQueue<Spans> spansQueue = new PriorityQueue<>(SPANS_COMPARATOR);
+    long cost;
+
+    void addSpans(Spans spans) {
+      if (spans.docID() < 0 || spans.docID() == NO_MORE_DOCS) {
+        throw new IllegalArgumentException("Expecting given spans to be in a positioned state.");
+      }
+      spansQueue.add(spans);
+      cost = Math.max(cost, spans.cost());
+    }
+
+    // DocIdSetIterator methods:
+
+    @Override
+    public int nextDoc() throws IOException {
+      if (spansQueue.isEmpty()) {
+        return NO_MORE_DOCS;
+      }
+      return advance(spansQueue.peek().docID() + 1);
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      if (spansQueue.isEmpty()) {
+        return NO_MORE_DOCS;
+      }
+      while (true) {
+        Spans spans = spansQueue.peek();
+        if (spans.docID() >= target) {
+          return spans.docID();
+        }
+        spansQueue.remove(); // must remove before modify state
+        if (spans.advance(target) != NO_MORE_DOCS) { // ... otherwise it's not re-added
+          spansQueue.add(spans);
+        } else if (spansQueue.isEmpty()) {
+          return NO_MORE_DOCS;
+        }
+      }
+    }
+
+    @Override
+    public int docID() {
+      if (spansQueue.isEmpty()) {
+        return NO_MORE_DOCS;
+      }
+      return spansQueue.peek().docID();
+    }
+
+    @Override
+    public long cost() {
+      return cost;
+    }
+
+    // Spans methods:
+
+    @Override
+    public int nextStartPosition() throws IOException {
+      // advance any spans at the initial position per document
+      boolean atDocStart = false;
+      while (spansQueue.peek().startPosition() == -1) {
+        atDocStart = true;
+        Spans headSpans = spansQueue.remove(); // remove because we will change state
+        headSpans.nextStartPosition();
+        spansQueue.add(headSpans);
+      }
+      if (!atDocStart) {
+        Spans headSpans = spansQueue.remove(); // remove because we will change state
+        headSpans.nextStartPosition();
+        spansQueue.add(headSpans);
+      }
+      return startPosition();
+    }
+
+    @Override
+    public int startPosition() {
+      return spansQueue.peek().startPosition();
+    }
+
+    @Override
+    public int endPosition() {
+      return spansQueue.peek().endPosition();
+    }
+
+    @Override
+    public int width() {
+      return spansQueue.peek().width();
+    }
+
+    @Override
+    public void collect(SpanCollector collector) throws IOException {
+      spansQueue.peek().collect(collector);
+    }
+
+    @Override
+    public float positionsCost() {
+      return 100f;// no idea; and we can't delegate due to not allowing to call it dependent on TwoPhaseIterator
+    }
+  }
+
+  /**
+   * A Spans based on a list of cached spans for one doc.  It is pre-positioned to this doc.
+   */
+  private static class CachedSpans extends Spans {
+
+    private static class CachedSpan {
+      final int start;
+      final int end;
+
+      CachedSpan(int start, int end) {
+        this.start = start;
+        this.end = end;
+      }
+    }
+
+    final int docId;
+    final ArrayList<CachedSpan> cachedSpanList;
+    int index = -1;
+
+    CachedSpans(Spans spans) throws IOException {
+      this.docId = spans.docID();
+      assert this.docId != -1;
+      // Consume the spans for this doc into a list.  There's always at least one; the first/current one.
+      cachedSpanList = new ArrayList<>();
+      while (spans.nextStartPosition() != NO_MORE_POSITIONS) {
+        cachedSpanList.add(new CachedSpan(spans.startPosition(), spans.endPosition()));
+      }
+      assert !cachedSpanList.isEmpty(); // bad Span impl?
+    }
+
+    /**
+     * Clone; reset iteration state.
+     */
+    CachedSpans(CachedSpans cloneMe) {
+      docId = cloneMe.docId;
+      cachedSpanList = cloneMe.cachedSpanList;
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      throw new UnsupportedOperationException("Not expected");
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      throw new UnsupportedOperationException("Not expected");
+    }
+
+    @Override
+    public int docID() {
+      return docId;
+    }
+
+    @Override
+    public long cost() {
+      return 1;
+    }
+
+    @Override
+    public int nextStartPosition() throws IOException {
+      index++;
+      return startPosition();
+    }
+
+    @Override
+    public int startPosition() {
+      return index < 0 ?
+          -1 : index >= cachedSpanList.size() ?
+          NO_MORE_POSITIONS : cachedSpanList.get(index).start;
+    }
+
+    @Override
+    public int endPosition() {
+      return index < 0 ?
+          -1 : index >= cachedSpanList.size() ?
+          NO_MORE_POSITIONS : cachedSpanList.get(index).end;
+    }
+
+    @Override
+    public int width() {
+      return endPosition() - startPosition();
+    }
+
+    @Override
+    public void collect(SpanCollector collector) throws IOException {
+      throw new UnsupportedOperationException("Not expected");
+    }
+
+    @Override
+    public float positionsCost() {
+      return 1f;
+    }
+
+  } // class CachedSpans
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/722e8271/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
new file mode 100644
index 0000000..4666906
--- /dev/null
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/PostingsOffsetStrategy.java
@@ -0,0 +1,61 @@
+/*
+ * 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.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
+ * does not support multi-term queries; the highlighter will fallback on analysis for that.
+ *
+ * @lucene.internal
+ */
+public class PostingsOffsetStrategy extends FieldOffsetStrategy {
+
+  public PostingsOffsetStrategy(String field, BytesRef[] queryTerms, PhraseHelper phraseHelper, CharacterRunAutomaton[] automata) {
+    super(field, queryTerms, phraseHelper, automata);
+  }
+
+  @Override
+  public List<OffsetsEnum> getOffsetsEnums(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 leaf reader
+    }
+
+    return createOffsetsEnumsFromReader(leafReader, docId);
+  }
+
+  @Override
+  public UnifiedHighlighter.OffsetSource getOffsetSource() {
+    return UnifiedHighlighter.OffsetSource.POSTINGS;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/722e8271/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
new file mode 100644
index 0000000..81de379
--- /dev/null
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/PostingsWithTermVectorsOffsetStrategy.java
@@ -0,0 +1,71 @@
+/*
+ * 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.analysis.TokenStream;
+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.
+ *
+ * @lucene.internal
+ */
+public class PostingsWithTermVectorsOffsetStrategy extends FieldOffsetStrategy {
+
+  public PostingsWithTermVectorsOffsetStrategy(String field, BytesRef[] queryTerms, PhraseHelper phraseHelper, CharacterRunAutomaton[] automata) {
+    super(field, queryTerms, phraseHelper, automata);
+  }
+
+  @Override
+  public List<OffsetsEnum> getOffsetsEnums(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);
+    if (docTerms == null) {
+      return Collections.emptyList();
+    }
+    leafReader = new TermVectorFilteredLeafReader(leafReader, docTerms);
+
+    TokenStream tokenStream = automata.length > 0 ? MultiTermHighlighting
+        .uninvertAndFilterTerms(leafReader.terms(field), docId, this.automata, content.length()) : null;
+
+    return createOffsetsEnums(leafReader, docId, tokenStream);
+  }
+
+  @Override
+  public UnifiedHighlighter.OffsetSource getOffsetSource() {
+    return UnifiedHighlighter.OffsetSource.POSTINGS;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/722e8271/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/SplittingBreakIterator.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/SplittingBreakIterator.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/SplittingBreakIterator.java
new file mode 100644
index 0000000..b3a415c
--- /dev/null
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/SplittingBreakIterator.java
@@ -0,0 +1,244 @@
+/*
+ * 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.text.BreakIterator;
+import java.text.CharacterIterator;
+import java.text.StringCharacterIterator;
+
+/**
+ * Virtually slices the text on both sides of every occurrence of the specified character. If the slice is 0-length
+ * which happens for adjacent slice characters or when they are at the beginning or end, that character is reported
+ * as a boundary.
+ * For every slice between the specified characters, it is further processed with a specified
+ * BreakIterator. A consequence is that the enclosed BreakIterator will never "see" the splitting character.
+ * <br>
+ * <em>Note: {@link #setText(CharacterIterator)} is unsupported. Use the string version.</em>
+ *
+ * @lucene.experimental
+ */
+public class SplittingBreakIterator extends BreakIterator {
+  private final BreakIterator baseIter;
+  private final char sliceChar;
+
+  private String text;
+  private int sliceStartIdx;
+  private int sliceEndIdx;
+  private int current;
+
+  public SplittingBreakIterator(BreakIterator baseIter, char sliceChar) {
+    this.baseIter = baseIter;
+    this.sliceChar = sliceChar;
+  }
+
+  @Override
+  public void setText(CharacterIterator newText) {
+    throw new UnsupportedOperationException("unexpected");
+  }
+
+  @Override
+  public void setText(String newText) {
+    this.text = newText;
+    first();
+  }
+
+  @Override
+  public CharacterIterator getText() {
+    StringCharacterIterator charIter = new StringCharacterIterator(text);
+    // API doesn't say what the state should be but it should probably be at the current index.
+    charIter.setIndex(current());
+    return charIter;
+  }
+
+  @Override
+  public int current() {
+    assert current != DONE;
+    return current; // MUST be updated by the other methods when result isn't DONE.
+  }
+
+  @Override
+  public int first() {
+    sliceStartIdx = 0;
+    sliceEndIdx = text.indexOf(sliceChar);
+    if (sliceEndIdx == -1) {
+      sliceEndIdx = text.length();
+    }
+    if (sliceStartIdx == sliceEndIdx) {
+      return current = sliceStartIdx;
+    }
+    baseIter.setText(text.substring(sliceStartIdx, sliceEndIdx));
+    return current = sliceStartIdx + baseIter.current();// since setText() sets to first(), just grab current()
+  }
+
+  @Override
+  public int last() {
+    sliceEndIdx = text.length();
+    sliceStartIdx = text.lastIndexOf(sliceChar);
+    if (sliceStartIdx == -1) {
+      sliceStartIdx = 0;
+    } else {
+      sliceStartIdx++;//past sliceChar
+    }
+    if (sliceEndIdx == sliceStartIdx) {
+      return current = sliceEndIdx;
+    }
+    baseIter.setText(text.substring(sliceStartIdx, sliceEndIdx));
+    return current = sliceStartIdx + baseIter.last();
+  }
+
+  @Override
+  public int next() {
+    int prevCurrent = current;
+    current = sliceStartIdx == sliceEndIdx ? DONE : baseIter.next();
+    if (current != DONE) {
+      return current = current + sliceStartIdx;
+    }
+    if (sliceEndIdx >= text.length()) {
+      current = prevCurrent;//keep current where it is
+      return DONE;
+    }
+    sliceStartIdx = sliceEndIdx + 1;
+    sliceEndIdx = text.indexOf(sliceChar, sliceStartIdx);
+    if (sliceEndIdx == -1) {
+      sliceEndIdx = text.length();
+    }
+    if (sliceStartIdx == sliceEndIdx) {
+      return current = sliceStartIdx;
+    }
+    baseIter.setText(text.substring(sliceStartIdx, sliceEndIdx));
+    return current = sliceStartIdx + baseIter.current();//use current() since at first() already
+  }
+
+  @Override
+  public int previous() { // note: closely follows next() but reversed
+    int prevCurrent = current;
+    current = sliceStartIdx == sliceEndIdx ? DONE : baseIter.previous();
+    if (current != DONE) {
+      return current = current + sliceStartIdx;
+    }
+    if (sliceStartIdx == 0) {
+      current = prevCurrent;//keep current where it is
+      return DONE;
+    }
+    sliceEndIdx = sliceStartIdx - 1;
+    sliceStartIdx = text.lastIndexOf(sliceChar, sliceEndIdx - 1);
+    if (sliceStartIdx == -1) {
+      sliceStartIdx = 0;
+    } else {
+      sliceStartIdx++;//past sliceChar
+    }
+    if (sliceStartIdx == sliceEndIdx) {
+      return current = sliceStartIdx;
+    }
+    baseIter.setText(text.substring(sliceStartIdx, sliceEndIdx));
+    return current = sliceStartIdx + baseIter.last();
+  }
+
+  @Override
+  public int following(int offset) {
+    // if the offset is not in this slice, update the slice
+    if (offset + 1 < sliceStartIdx || offset + 1 > sliceEndIdx) {
+      if (offset == text.length()) { // DONE condition
+        last(); // because https://bugs.openjdk.java.net/browse/JDK-8015110
+        return DONE;
+      }
+      sliceStartIdx = text.lastIndexOf(sliceChar, offset);//no +1
+      if (sliceStartIdx == -1) {
+        sliceStartIdx = 0;
+      } else {
+        sliceStartIdx++;//move past separator
+      }
+      sliceEndIdx = text.indexOf(sliceChar, Math.max(offset + 1, sliceStartIdx));
+      if (sliceEndIdx == -1) {
+        sliceEndIdx = text.length();
+      }
+      if (sliceStartIdx != sliceEndIdx) {//otherwise, adjacent separator or separator at end
+        baseIter.setText(text.substring(sliceStartIdx, sliceEndIdx));
+      }
+    }
+
+    // lookup following() in this slice:
+    if (sliceStartIdx == sliceEndIdx) {
+      return current = offset + 1;
+    } else {
+      // note: following() can never be first() if the first character is a boundary (it usually is).
+      //   So we have to check if we should call first() instead of following():
+      if (offset == sliceStartIdx - 1) {
+        // the first boundary following this offset is the very first boundary in this slice
+        return current = sliceStartIdx + baseIter.first();
+      } else {
+        return current = sliceStartIdx + baseIter.following(offset - sliceStartIdx);
+      }
+    }
+  }
+
+  @Override
+  public int preceding(int offset) { // note: closely follows following() but reversed
+    if (offset - 1 < sliceStartIdx || offset - 1 > sliceEndIdx) {
+      if (offset == 0) { // DONE condition
+        first(); // because https://bugs.openjdk.java.net/browse/JDK-8015110
+        return DONE;
+      }
+      sliceEndIdx = text.indexOf(sliceChar, offset);//no -1
+      if (sliceEndIdx == -1) {
+        sliceEndIdx = text.length();
+      }
+      sliceStartIdx = text.lastIndexOf(sliceChar, offset - 1);
+      if (sliceStartIdx == -1) {
+        sliceStartIdx = 0;
+      } else {
+        sliceStartIdx = Math.min(sliceStartIdx + 1, sliceEndIdx);
+      }
+      if (sliceStartIdx != sliceEndIdx) {//otherwise, adjacent separator or separator at end
+        baseIter.setText(text.substring(sliceStartIdx, sliceEndIdx));
+      }
+    }
+    // lookup preceding() in this slice:
+    if (sliceStartIdx == sliceEndIdx) {
+      return current = offset - 1;
+    } else {
+      // note: preceding() can never be last() if the last character is a boundary (it usually is).
+      //   So we have to check if we should call last() instead of preceding():
+      if (offset == sliceEndIdx + 1) {
+        // the last boundary preceding this offset is the very last boundary in this slice
+        return current = sliceStartIdx + baseIter.last();
+      } else {
+        return current = sliceStartIdx + baseIter.preceding(offset - sliceStartIdx);
+      }
+    }
+  }
+
+  @Override
+  public int next(int n) {
+    if (n < 0) {
+      for (int i = 0; i < -n; i++) {
+        if (previous() == DONE) {
+          return DONE;
+        }
+      }
+    } else {
+      for (int i = 0; i < n; i++) {
+        if (next() == DONE) {
+          return DONE;
+        }
+      }
+    }
+    return current();
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/722e8271/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
new file mode 100644
index 0000000..954024c
--- /dev/null
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/TermVectorFilteredLeafReader.java
@@ -0,0 +1,130 @@
+/*
+ * 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.Fields;
+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;
+
+/**
+ * A filtered LeafReader that only includes the terms that are also in a provided set of terms.
+ * Certain methods may be unimplemented or cause large operations on the underlying reader
+ * and be slow.
+ *
+ * @lucene.internal
+ */
+final class TermVectorFilteredLeafReader extends FilterLeafReader {
+  // NOTE: super ("in") is baseLeafReader
+
+  private final Terms filterTerms;
+
+  /**
+   * <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.
+   */
+  TermVectorFilteredLeafReader(LeafReader baseLeafReader, Terms filterTerms) {
+    super(baseLeafReader);
+    this.filterTerms = filterTerms;
+  }
+
+  @Override
+  public Fields fields() throws IOException {
+    return new TermVectorFilteredFields(in.fields(), filterTerms);
+  }
+
+  private static final class TermVectorFilteredFields extends FilterLeafReader.FilterFields {
+    // NOTE: super ("in") is baseFields
+
+    private final Terms filterTerms;
+
+    TermVectorFilteredFields(Fields baseFields, Terms filterTerms) {
+      super(baseFields);
+      this.filterTerms = filterTerms;
+    }
+
+    @Override
+    public Terms terms(String field) throws IOException {
+      return new TermsFilteredTerms(in.terms(field), filterTerms);
+    }
+  }
+
+  private static final class TermsFilteredTerms extends FilterLeafReader.FilterTerms {
+    // NOTE: super ("in") is the baseTerms
+
+    private final Terms filterTerms;
+
+    TermsFilteredTerms(Terms baseTerms, Terms filterTerms) {
+      super(baseTerms);
+      this.filterTerms = filterTerms;
+    }
+
+    //TODO delegate size() ?
+
+    //TODO delegate getMin, getMax to filterTerms
+
+    @Override
+    public TermsEnum iterator() throws IOException {
+      return new TermVectorFilteredTermsEnum(in.iterator(), filterTerms.iterator());
+    }
+
+    @Override
+    public TermsEnum intersect(CompiledAutomaton compiled, BytesRef startTerm) throws IOException {
+      return new TermVectorFilteredTermsEnum(in.iterator(), filterTerms.intersect(compiled, startTerm));
+    }
+  }
+
+  private static final class TermVectorFilteredTermsEnum extends FilterLeafReader.FilterTermsEnum {
+    // NOTE: super ("in") is the filteredTermsEnum. This is different than wrappers above because we
+    //    navigate the terms using the filter.
+
+    //TODO: track the last term state from the term state method and do some potential optimizations
+    private final TermsEnum baseTermsEnum;
+
+    TermVectorFilteredTermsEnum(TermsEnum baseTermsEnum, TermsEnum filteredTermsEnum) {
+      super(filteredTermsEnum); // note this is reversed from constructors above
+      this.baseTermsEnum = baseTermsEnum;
+    }
+
+    //TODO delegate docFreq & ttf (moveToCurrentTerm() then call on full?
+
+    @Override
+    public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
+      moveToCurrentTerm();
+      return baseTermsEnum.postings(reuse, flags);
+    }
+
+    void moveToCurrentTerm() throws IOException {
+      BytesRef currentTerm = in.term(); // from filteredTermsEnum
+      boolean termInBothTermsEnum = baseTermsEnum.seekExact(currentTerm);
+
+      if (!termInBothTermsEnum) {
+        throw new IllegalStateException("Term vector term " + currentTerm + " does not appear in full index.");
+      }
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/722e8271/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
new file mode 100644
index 0000000..204679b
--- /dev/null
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/TermVectorOffsetStrategy.java
@@ -0,0 +1,68 @@
+/*
+ * 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.analysis.TokenStream;
+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.
+ *
+ * @lucene.internal
+ */
+public class TermVectorOffsetStrategy extends FieldOffsetStrategy {
+
+  public TermVectorOffsetStrategy(String field, BytesRef[] queryTerms, PhraseHelper phraseHelper, CharacterRunAutomaton[] automata) {
+    super(field, queryTerms, phraseHelper, automata);
+  }
+
+  @Override
+  public UnifiedHighlighter.OffsetSource getOffsetSource() {
+    return UnifiedHighlighter.OffsetSource.TERM_VECTORS;
+  }
+
+  @Override
+  public List<OffsetsEnum> getOffsetsEnums(IndexReader reader, int docId, String content) throws IOException {
+    Terms tvTerms = reader.getTermVector(docId, field);
+    if (tvTerms == null) {
+      return Collections.emptyList();
+    }
+
+    LeafReader leafReader = null;
+    if ((terms.length > 0) || strictPhrases.willRewrite()) {
+      leafReader = new TermVectorLeafReader(field, tvTerms);
+      docId = 0;
+    }
+
+    TokenStream tokenStream = null;
+    if (automata.length > 0) {
+      tokenStream = MultiTermHighlighting.uninvertAndFilterTerms(tvTerms, 0, automata, content.length());
+    }
+
+    return createOffsetsEnums(leafReader, docId, tokenStream);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/722e8271/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/TokenStreamFromTermVector.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/TokenStreamFromTermVector.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/TokenStreamFromTermVector.java
new file mode 100644
index 0000000..980c566
--- /dev/null
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/TokenStreamFromTermVector.java
@@ -0,0 +1,395 @@
+/*
+ * 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.Collections;
+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.analysis.tokenattributes.PayloadAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+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.BytesRefArray;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.CharsRefBuilder;
+import org.apache.lucene.util.Counter;
+import org.apache.lucene.util.UnicodeUtil;
+
+/**
+ * TokenStream created from a term vector field. The term vector requires positions and/or offsets (either). If you
+ * want payloads add PayloadAttributeImpl (as you would normally) but don't assume the attribute is already added just
+ * because you know the term vector has payloads, since the first call to incrementToken() will observe if you asked
+ * for them and if not then won't get them.  This TokenStream supports an efficient {@link #reset()}, so there's
+ * no need to wrap with a caching impl.
+ *
+ * @lucene.internal
+ */
+final class TokenStreamFromTermVector extends TokenStream {
+  // note: differs from similar class in the standard highlighter. This one is optimized for sparse cases.
+
+  /**
+   * content length divided by distinct positions; an average of dense text.
+   */
+  private static final double AVG_CHARS_PER_POSITION = 6;
+
+  private static final int INSERTION_SORT_THRESHOLD = 16;
+
+  private final Terms vector;
+
+  private final int filteredDocId;
+
+  private final CharTermAttribute termAttribute;
+
+  private final PositionIncrementAttribute positionIncrementAttribute;
+
+  private final int offsetLength;
+
+  private final float loadFactor;
+
+  private OffsetAttribute offsetAttribute;//maybe null
+
+  private PayloadAttribute payloadAttribute;//maybe null
+
+  private CharsRefBuilder termCharsBuilder;//term data here
+
+  private BytesRefArray payloadsBytesRefArray;//only used when payloadAttribute is non-null
+  private BytesRefBuilder spareBytesRefBuilder;//only used when payloadAttribute is non-null
+
+  private TokenLL firstToken = null; // the head of a linked-list
+
+  private TokenLL incrementToken = null;
+
+  private boolean initialized = false;//lazy
+
+  public TokenStreamFromTermVector(Terms vector, int offsetLength) throws IOException {
+    this(vector, 0, offsetLength, 1f);
+  }
+
+  /**
+   * Constructor.
+   *
+   * @param vector        Terms that contains the data for
+   *                      creating the TokenStream. Must have positions and/or offsets.
+   * @param filteredDocId The docID we will process.
+   * @param offsetLength  Supply the character length of the text being uninverted, or a lower value if you don't want
+   *                      to invert text beyond an offset (in so doing this will act as a filter).  If you don't
+   *                      know the length, pass -1.  In conjunction with {@code loadFactor}, it's used to
+   *                      determine how many buckets to create during uninversion.
+   *                      It's also used to filter out tokens with a start offset exceeding this value.
+   * @param loadFactor    The percent of tokens from the original terms (by position count) that are
+   *                      expected to be inverted.  If they are filtered (e.g.
+   *                      {@link org.apache.lucene.index.FilterLeafReader.FilterTerms})
+   *                      then consider using less than 1.0 to avoid wasting space.
+   *                      1.0 means all, 1/64th would suggest 1/64th of all tokens coming from vector.
+   */
+  TokenStreamFromTermVector(Terms vector, int filteredDocId, int offsetLength, float loadFactor) throws IOException {
+    super();
+    this.filteredDocId = filteredDocId;
+    this.offsetLength = offsetLength == Integer.MAX_VALUE ? -1 : offsetLength;
+    if (loadFactor <= 0f || loadFactor > 1f) {
+      throw new IllegalArgumentException("loadFactor should be > 0 and <= 1");
+    }
+    this.loadFactor = loadFactor;
+    assert !hasAttribute(PayloadAttribute.class) : "AttributeFactory shouldn't have payloads *yet*";
+    if (!vector.hasPositions() && !vector.hasOffsets()) {
+      throw new IllegalArgumentException("The term vector needs positions and/or offsets.");
+    }
+    assert vector.hasFreqs();
+    this.vector = vector;
+    termAttribute = addAttribute(CharTermAttribute.class);
+    positionIncrementAttribute = addAttribute(PositionIncrementAttribute.class);
+  }
+
+  public Terms getTermVectorTerms() {
+    return vector;
+  }
+
+  @Override
+  public void reset() throws IOException {
+    incrementToken = null;
+    super.reset();
+  }
+
+  //We delay initialization because we can see which attributes the consumer wants, particularly payloads
+  private void init() throws IOException {
+    assert !initialized;
+    int dpEnumFlags = 0;
+    if (vector.hasOffsets()) {
+      offsetAttribute = addAttribute(OffsetAttribute.class);
+      dpEnumFlags |= PostingsEnum.OFFSETS;
+    }
+    if (vector.hasPayloads() && hasAttribute(PayloadAttribute.class)) {
+      payloadAttribute = getAttribute(PayloadAttribute.class);
+      payloadsBytesRefArray = new BytesRefArray(Counter.newCounter());
+      spareBytesRefBuilder = new BytesRefBuilder();
+      dpEnumFlags |= PostingsEnum.PAYLOADS;
+    }
+
+    // We put term data here
+    termCharsBuilder = new CharsRefBuilder();
+    termCharsBuilder.grow(initTotalTermCharLen());
+
+    // Step 1: iterate termsEnum and create a token, placing into a bucketed array (given a load factor)
+
+    final TokenLL[] tokenBuckets = initTokenBucketsArray();
+    final double OFFSET_TO_BUCKET_IDX = loadFactor / AVG_CHARS_PER_POSITION;
+    final double POSITION_TO_BUCKET_IDX = loadFactor;
+
+    final TermsEnum termsEnum = vector.iterator();
+    BytesRef termBytesRef;
+    PostingsEnum dpEnum = null;
+    final CharsRefBuilder tempCharsRefBuilder = new CharsRefBuilder();//only for UTF8->UTF16 call
+
+    TERM_LOOP:
+    while ((termBytesRef = termsEnum.next()) != null) {
+      //Grab the term (in same way as BytesRef.utf8ToString() but we don't want a String obj)
+      // note: if term vectors supported seek by ord then we might just keep an int and seek by ord on-demand
+      tempCharsRefBuilder.grow(termBytesRef.length);
+      final int termCharsLen = UnicodeUtil.UTF8toUTF16(termBytesRef, tempCharsRefBuilder.chars());
+      final int termCharsOff = termCharsBuilder.length();
+      termCharsBuilder.append(tempCharsRefBuilder.chars(), 0, termCharsLen);
+      dpEnum = termsEnum.postings(dpEnum, dpEnumFlags);
+      assert dpEnum != null; // presumably checked by TokenSources.hasPositions earlier
+      int currentDocId = dpEnum.advance(filteredDocId);
+      if (currentDocId != filteredDocId) {
+        continue; //Not expected
+      }
+      final int freq = dpEnum.freq();
+      for (int j = 0; j < freq; j++) {
+        TokenLL token = new TokenLL();
+        token.position = dpEnum.nextPosition(); // can be -1 if not in the TV
+        token.termCharsOff = termCharsOff;
+        token.termCharsLen = (short) Math.min(termCharsLen, Short.MAX_VALUE);
+        // copy offset (if it's there) and compute bucketIdx
+        int bucketIdx;
+        if (offsetAttribute != null) {
+          token.startOffset = dpEnum.startOffset();
+          if (offsetLength >= 0 && token.startOffset > offsetLength) {
+            continue TERM_LOOP;//filter this token out; exceeds threshold
+          }
+          token.endOffsetInc = (short) Math.min(dpEnum.endOffset() - token.startOffset, Short.MAX_VALUE);
+          bucketIdx = (int) (token.startOffset * OFFSET_TO_BUCKET_IDX);
+        } else {
+          bucketIdx = (int) (token.position * POSITION_TO_BUCKET_IDX);
+        }
+        if (bucketIdx >= tokenBuckets.length) {
+          bucketIdx = tokenBuckets.length - 1;
+        }
+
+        if (payloadAttribute != null) {
+          final BytesRef payload = dpEnum.getPayload();
+          token.payloadIndex = payload == null ? -1 : payloadsBytesRefArray.append(payload);
+        }
+
+        //Add token to the head of the bucket linked list
+        token.next = tokenBuckets[bucketIdx];
+        tokenBuckets[bucketIdx] = token;
+      }
+    }
+
+    // Step 2:  Link all Tokens into a linked-list and sort all tokens at the same position
+
+    firstToken = initLinkAndSortTokens(tokenBuckets);
+
+    // If the term vector didn't have positions, synthesize them
+    if (!vector.hasPositions() && firstToken != null) {
+      TokenLL prevToken = firstToken;
+      prevToken.position = 0;
+      for (TokenLL token = prevToken.next; token != null; prevToken = token, token = token.next) {
+        if (prevToken.startOffset == token.startOffset) {
+          token.position = prevToken.position;
+        } else {
+          token.position = prevToken.position + 1;
+        }
+      }
+    }
+
+    initialized = true;
+  }
+
+  private static TokenLL initLinkAndSortTokens(TokenLL[] tokenBuckets) {
+    TokenLL firstToken = null;
+    List<TokenLL> scratchTokenArray = new ArrayList<>(); // declare here for re-use.  TODO use native array
+    TokenLL prevToken = null;
+    for (TokenLL tokenHead : tokenBuckets) {
+      if (tokenHead == null) {
+        continue;
+      }
+      //sort tokens at this position and link them; return the first
+      TokenLL tokenTail;
+      // just one token
+      if (tokenHead.next == null) {
+        tokenTail = tokenHead;
+      } else {
+        // add the linked list to a temporary array
+        for (TokenLL cur = tokenHead; cur != null; cur = cur.next) {
+          scratchTokenArray.add(cur);
+        }
+        // sort; and set tokenHead & tokenTail
+        if (scratchTokenArray.size() < INSERTION_SORT_THRESHOLD) {
+          // insertion sort by creating a linked list (leave scratchTokenArray alone)
+          tokenHead = tokenTail = scratchTokenArray.get(0);
+          tokenHead.next = null;
+          for (int i = 1; i < scratchTokenArray.size(); i++) {
+            TokenLL insertToken = scratchTokenArray.get(i);
+            if (insertToken.compareTo(tokenHead) <= 0) {
+              // takes the place of tokenHead
+              insertToken.next = tokenHead;
+              tokenHead = insertToken;
+            } else {
+              // goes somewhere after tokenHead
+              for (TokenLL prev = tokenHead; true; prev = prev.next) {
+                if (prev.next == null || insertToken.compareTo(prev.next) <= 0) {
+                  if (prev.next == null) {
+                    tokenTail = insertToken;
+                  }
+                  insertToken.next = prev.next;
+                  prev.next = insertToken;
+                  break;
+                }
+              }
+            }
+          }
+        } else {
+          Collections.sort(scratchTokenArray);
+          // take back out and create a linked list
+          TokenLL prev = tokenHead = scratchTokenArray.get(0);
+          for (int i = 1; i < scratchTokenArray.size(); i++) {
+            prev.next = scratchTokenArray.get(i);
+            prev = prev.next;
+          }
+          tokenTail = prev;
+          tokenTail.next = null;
+        }
+        scratchTokenArray.clear();//too bad ArrayList nulls it out; we don't actually need that
+      }
+
+      //link to previous
+      if (prevToken != null) {
+        assert prevToken.next == null;
+        prevToken.next = tokenHead; //concatenate linked-list
+        assert prevToken.compareTo(tokenHead) < 0 : "wrong offset / position ordering expectations";
+      } else {
+        assert firstToken == null;
+        firstToken = tokenHead;
+      }
+
+      prevToken = tokenTail;
+    }
+    return firstToken;
+  }
+
+  private int initTotalTermCharLen() throws IOException {
+    int guessNumTerms;
+    if (vector.size() != -1) {
+      guessNumTerms = (int) vector.size();
+    } else if (offsetLength != -1) {
+      guessNumTerms = (int) (offsetLength * 0.33);//guess 1/3rd
+    } else {
+      return 128;
+    }
+    return Math.max(64, (int) (guessNumTerms * loadFactor * 7.0));//7 is over-estimate of average term len
+  }
+
+  private TokenLL[] initTokenBucketsArray() throws IOException {
+    // Estimate the number of non-empty positions (number of tokens, excluding same-position synonyms).
+    int positionsEstimate;
+    if (offsetLength == -1) { // no clue what the char length is.
+      // Estimate the number of position slots we need from term stats based on Wikipedia.
+      int sumTotalTermFreq = (int) vector.getSumTotalTermFreq();
+      if (sumTotalTermFreq == -1) {//unfortunately term vectors seem to not have this stat
+        int size = (int) vector.size();
+        if (size == -1) {//doesn't happen with term vectors, it seems, but pick a default any way
+          size = 128;
+        }
+        sumTotalTermFreq = (int) (size * 2.4);
+      }
+      positionsEstimate = (int) (sumTotalTermFreq * 1.5);//less than 1 in 10 docs exceed this
+    } else {
+      // guess number of token positions by this factor.
+      positionsEstimate = (int) (offsetLength / AVG_CHARS_PER_POSITION);
+    }
+    // apply the load factor.
+    return new TokenLL[Math.max(1, (int) (positionsEstimate * loadFactor))];
+  }
+
+  @Override
+  public boolean incrementToken() throws IOException {
+    int posInc;
+    if (incrementToken == null) {
+      if (!initialized) {
+        init();
+        assert initialized;
+      }
+      incrementToken = firstToken;
+      if (incrementToken == null) {
+        return false;
+      }
+      posInc = incrementToken.position + 1;//first token normally has pos 0; add 1 to get posInc
+    } else if (incrementToken.next != null) {
+      int lastPosition = incrementToken.position;
+      incrementToken = incrementToken.next;
+      posInc = incrementToken.position - lastPosition;
+    } else {
+      return false;
+    }
+    clearAttributes();
+    termAttribute.copyBuffer(termCharsBuilder.chars(), incrementToken.termCharsOff, incrementToken.termCharsLen);
+
+    positionIncrementAttribute.setPositionIncrement(posInc);
+    if (offsetAttribute != null) {
+      offsetAttribute.setOffset(incrementToken.startOffset, incrementToken.startOffset + incrementToken.endOffsetInc);
+    }
+    if (payloadAttribute != null && incrementToken.payloadIndex >= 0) {
+      payloadAttribute.setPayload(payloadsBytesRefArray.get(spareBytesRefBuilder, incrementToken.payloadIndex));
+    }
+    return true;
+  }
+
+  private static class TokenLL implements Comparable<TokenLL> {
+    // This class should weigh 32 bytes, including object header
+
+    int termCharsOff; // see termCharsBuilder
+    short termCharsLen;
+
+    int position;
+    int startOffset;
+    short endOffsetInc; // add to startOffset to get endOffset
+    int payloadIndex;
+
+    TokenLL next;
+
+    @Override
+    public int compareTo(TokenLL tokenB) {
+      int cmp = Integer.compare(this.position, tokenB.position);
+      if (cmp == 0) {
+        cmp = Integer.compare(this.startOffset, tokenB.startOffset);
+        if (cmp == 0) {
+          cmp = Short.compare(this.endOffsetInc, tokenB.endOffsetInc);
+        }
+      }
+      return cmp;
+    }
+  }
+}